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/03/23 12:38:01 UTC

[GitHub] [airflow] potiuk opened a new pull request #7832: [WIP] Add production image support

potiuk opened a new pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832
 
 
   ---
   Issue link: WILL BE INSERTED BY [boring-cyborg](https://github.com/kaxil/boring-cyborg)
   
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Unit tests coverage for changes (not needed for documentation changes)
   - [x] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [x] Relevant documentation is updated including usage instructions.
   - [x] I will engage committers as explained in [Contribution Workflow Example](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#contribution-workflow-example).
   
   ---
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   Read the [Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines) for more information.
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401581129
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -263,8 +310,8 @@ to enter the running container. It's as easy as launching ``breeze exec`` while
 Breeze environment. You will be dropped into bash and environment variables will be read in the same
 way as when you enter the environment. You can do it multiple times and open as many terminals as you need.
 
-Stopping Breeze
----------------
+Stopping Interactive environment
 
 Review comment:
   Thks!
   

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400872913
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
+
+Those variables should be overwritten only if the production image is built from local sources
+rather than from GitHub repository.
+
+* COPY_SOURCE - should be set to "." if you the image is built from sources
+* COPY_TARGET - should be set to a airflow source directory inside container if the image is built
+  from sources
+* AIRFLOW_SOURCES - should point to the same as COPY_TARGET in case production image is installed from
+  the local sources rather than from Github repository. If left empty it points to sources from
+  GitHub repository derived from AIRFLOW_ORG, AIRFLOW_REPO, AIRFLOW_GIT_REFERENCE
+* CONSTRAINT_REQUIREMENTS should point to locally available requirements file if the image is built
+  from sources. If left empty it will point to the right requirement file from GitHub repository
+  derived from AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+* ENTRYPOINT_FILE - sources point to entrypoint.sh file when installing from sources if left empty
+  it points to sources from GitHub repository derived from
+  AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+
+The below will production image in version 3.6 with default extras from master branch in Github:
+
+.. code-block::
+
+  docker build .
+
+The below will build the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github. Note that versions 1.10.9 and below
+have no requirements so requirements should be taken from head of the v1-10-test branch. Once we
+release 1.10.10 we can take them from the 1.10.10 tag.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg AIRFLOW_GIT_REFERENCE=1.10.9 \
+    --build-arg REQUIREMENTS_GIT_REFERENCE=v1-10-test --build-arg WWW_FOLDER="www_rbac"
+
+The below will build the production image in version 3.7 with default extras from master branch in Github.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+The below will build the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE=entrypoint.sh
+
+Image manifests
+---------------
+
+Together with teh main images we also build an push image manifests. Those manifests are vey small images
+that contain only results of the docker inspect for the image. This is in order to be able to
+determine very quickly if the image in the docker registry has changed a lot since the last time.
+Unfortunately docker registry (specifically dockerhub registry) has no anonymous way of querying image
+details via API, you need to download the image to inspect it. We overcame it in the way that
+always when we build the image we build a very small image manifest and push it to registry together
+with the main image. The tag for the manifest image is the same as for the image it refers
+to with added ``-manifest`` suffix. For example manifest image for ``apache/airflow:1.10.10`` image is named
+``apache/airflow:1.10.10-manifest``. Similarly ``apache/airflow:master-ci`` manifest is named
+``apache/airflow:master-ci-manifest``.
+
+
+Pulling the Latest Images
+-------------------------
+
+Sometimes the image needs to be rebuilt from scratch. This is required, for example,
+when there is a security update of the Python version that all the images are based on and new version
+of the image is pushed to the repository. In this case it is usually faster to pull the latest
+images rather than rebuild them from scratch.
+
+You can do it via the ``--force-pull-images`` flag to force pulling the latest images from the Docker Hub.
+
+For production image:
+
+.. code-block::
+
+  ./breeze build-image --force-pull-images --production-image
+
+For CI image Breeze automatically uses force pulling in case it determines that your image is very outdated,
+however uou can also force it with the same flag.
+
+.. code-block::
+
+  ./breeze build-image --force-pull-images
+
+Using the images
+================
+
+Both images have entrypoint set as dumb-init with entrypoint.sh script executed (in order to forward
+signals). This entrypoint works as follows:
+
+* If AIRFLOW__CORE__SQL_ALCHEMY_CONN variable is passed to the container and it is either mysql or postgres
 
 Review comment:
   ```suggestion
   * If ``AIRFLOW__CORE__SQL_ALCHEMY_CONN`` variable is passed to the container and it is either mysql or postgres
   ```
   
   Can you wrap all those args/env vars with backticks please

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401511282
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
 
 Review comment:
   Sure. Will do. I am planning to revert the logic. By default I will use source to build. But via providing arguments we will be able to build it from GIT or from released version.

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402503373
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
 
 Review comment:
   Feel free to create a PR for these fixes :)

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.26%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   86.79%   +0.26%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39353     +146     
   + Misses       6105     5989     -116     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [31 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...602a289](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402518544
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
 
 Review comment:
   Fixed in new PR coming.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.64%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   87.17%   +0.64%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39526     +319     
   + Misses       6105     5816     -289     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [35 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...3b336c6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399158985
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -348,18 +349,43 @@ In case of disk space errors on macOS, increase the disk space available for Doc
 Building the Images
 -------------------
 
-You can manually trigger building the local images using the script:
+You can manually trigger building the local CI image using this command:
 
 .. code-block::
 
-  ./breeze build-only
+  ./breeze build-ci-image
 
-The scripts that build the images are optimized to minimize the time needed to rebuild the image when
+And production image using this command:
+
+.. code-block::
+
+  ./breeze build-prod-image
+
+
+The images are build with default extras - different for CI image and different for production image
+and using python version chosen in breeze. You can change those parameters by specifying them as
+command line options. You can see default extras used via  ``./breeze flags`` or in the syntax example
+below.
+
+For example if you want to build python 3.7 version of production image with
+all extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-prod-image --python 3.7 --extras "all"
 
 Review comment:
   Is it the only option? Will `docker build` work too?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396702435
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
 
 Review comment:
   I am going to backport to 1.10.10
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400940955
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
 
 Review comment:
   It's markdown not RST but I can convert them easily :)

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.49%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   86.67%   -0.50%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39168     -223     
   - Misses       5796     6019     +223     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [4 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401825493
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            DETECTED_DB_USER=${BASH_REMATCH[2]} &&
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            DETECTED_DB_SCHEMA=${BASH_REMATCH[5]} &&
+            DETECTED_DB_PARAMS=${BASH_REMATCH[6]}
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW__CORE__SQL_ALCHEMY_CONN} == sqlite* ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|webserver)$ ]] || \
+        [[ ${DB_RESET:="false"} == "true" ]]; then
+    if [[ ${AIRFLOW_VERSION} == *1.10* ]]; then
 
 Review comment:
   Indeed. Good one. I would still want to keep it as I would like to use the same Dockerfile/entrypoint.sh in both master and v1-10-test without having to change anything when backporting. This line allows that.

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400868783
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
 
 Review comment:
   ```suggestion
   * ``BRANCH_OR_TAG`` - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/26b05bfe36d131ab19744514f9c9c69e9df126aa?src=pr&el=desc) will **increase** coverage by `0.09%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&token=WdLKlKHOAU&height=150&src=pr)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   87.16%   87.26%   +0.09%     
   ==========================================
     Files         932      932              
     Lines       45331    45331              
   ==========================================
   + Hits        39514    39559      +45     
   + Misses       5817     5772      -45
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/jobs/scheduler\_job.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL3NjaGVkdWxlcl9qb2IucHk=) | `90.94% <0%> (+0.57%)` | :arrow_up: |
   | [airflow/jobs/backfill\_job.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9qb2JzL2JhY2tmaWxsX2pvYi5weQ==) | `91.64% <0%> (+1.11%)` | :arrow_up: |
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0%> (+4.89%)` | :arrow_up: |
   | [airflow/executors/sequential\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvc2VxdWVudGlhbF9leGVjdXRvci5weQ==) | `100% <0%> (+44%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [26b05bf...182a141](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396946424
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
 
 Review comment:
   Yes. See the comment above just before the second "FROM". 
   
   We have two images (multi-segment):
   
   * The "build" image is there to run "pip install" - it needs MySQL to install the MySQL dev library and pip dependencies. But we only use that image to run the "pip install --user" and store the dependencies and libraries compiled during install in "${HOME}/.local"
   
   * The actual Airflow image: this one has no "build" dependencies (build-essentials) and during optimization we can remove few more unneeded libraries. Then from the "build" image we ONLY take the ".local" directory (this contains all python pip-insttaled dependencies and all binary-compiled .so libraries needed. This way the resulting Airflow image is much smaller. 
   
   We need the MySQL libraries in Airflow image because ... we want to connect to MySQL :).

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396456147
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
 
 Review comment:
   Are you talking about python-selinux? I have no idea yet who uses it (maybe you can figure it out?). Happy if you can do investigation on that one. I took this from the original CI image and it's likely it can be optimised away from there as well.
   
   Would you like to help with investigating this ?
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400625164
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_SOURCES}[${AIRFLOW_EXTRAS}]" \
+    --constraint /requirements.txt
+
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
 
 Review comment:
   Correct. Removed. Good eye!

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401517743
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
 
 Review comment:
   +1 

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.26%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   86.79%   +0.26%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39353     +146     
   + Misses       6105     5989     -116     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [31 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...602a289](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402412207
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   I think let's leave it as it is for now. We can decide on it and update it later to unblock 1.10.10RC

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402501422
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
+    --build-arg WWW_FOLDER="www_rbac"
+
+This builds the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE=entrypoint.sh \
+    --build-arg AIRFLOW_INSTALL_SOURCES="apache-airflow" \
+    --build-arg AIRFLOW_INSTALL_VERSION="==1.10.10" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt"
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
+    --build-arg WWW_FOLDER="www_rbac"
 
 Review comment:
   There is sth. wrong.
   
   ```suggestion
     docker build . 
       --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
       --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 
       --build-arg AIRFLOW_INSTALL_SOURCES="apache-airflow" \
       --build-arg AIRFLOW_INSTALL_VERSION="==1.10.10" \
       --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
       --build-arg ENTRYPOINT_FILE=entrypoint.sh \
       --build-arg AIRFLOW_SOURCES_FROM="Dockerfile" \
       --build-arg AIRFLOW_SOURCES_TO="/Dockerfile" \
       --build-arg WWW_FOLDER="www_rbac"
   ```
   
   Like this @potiuk ?

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400860902
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
 
 Review comment:
   ```suggestion
   #                        much smaller.
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400503159
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
 
 Review comment:
   I don't get what this is for? Why do we need the Dockerfile in the image at all?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399788815
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
 
 Review comment:
   Removed. looks good.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk edited a comment on issue #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on issue #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605626827
 
 
   > > /Users/kamilbregula/devel/google-airflow/airflow/scripts/ci/_utils.sh: line 628: AIRFLOW_REMOTE_MANIFEST_IMAGE: unbound variable
   
   Fixed.
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401195975
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
 
 Review comment:
   Official docker images are so you can do `docker pull airflow`? I'm not sure AFS would like us to drop the Apache name there.

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


With regards,
Apache Git Services

[GitHub] [airflow] dimberman commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
dimberman commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607978692
 
 
   @feluelle works! had to run `./breeze setup-autocomplete` and then restart shell. Might be worth mentioning in the docs or UPDATING.md?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-608021302
 
 
   > @feluelle works! had to run `./breeze setup-autocomplete` and then restart shell. Might be worth mentioning in the docs or UPDATING.md?
   
   I think it really depend on your setup. simply '. ./breeze-complete' should do. Did you have another directory with airflow somewhere? Tejh setup-autocomplete sets up a symbolic link to breeze-complete and only one can be active at the same time ... It's a bit tricky to get it right if you have more than one breeze command... The command itself will work though (it will use correct definition). I keep the master v1_10_test in sync so most of the auto-complete is good for both (some values like python version are not) and eventually you will get it right next time you get into the shell...
   
   Not sure if UPDATING.md is for those dev tools though.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402323779
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,333 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
 
 Review comment:
   If we do `docker build --build-arg AIRFLOW_INSTALL_SOURCES='apache-airflow'` to install form pip, then there won't be a package.json, so we should probably wrap this in an `[ -e "${AIRFLOW_WWW}/package.json ]`

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402352209
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,333 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
 
 Review comment:
   I checked how it is in Breeze - it is only needed in case we have different entrypoint.sh for different versions of Airflow. I am using the same Dockerfile to build 2.0 and 1.10 when installing from pypi or URL. for now there is no need indeed as I removed resetdb vs. dbreset -- so entrypoints will be the same but I want to keep the possibility that entrypoints will be different in 1.10 and 2.0 and that we would like to keep them separated. I really like (and yes) all the functionality to be able to build and run production-version of the released airflow in the master branch. That's really helpful to test 1.10 images.
   
   It's the same thing as the requirements.txt - we get them from v1_10_test branch in order to build 1.10 image from master.  I think it's rather useful. I found it useful for my testing at least
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401201166
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
 
 Review comment:
   This is quite an important point, and should be discussed on the mailing list.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401200600
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
 
 Review comment:
   It is? I read it and nothing there told me why we'd want the Dockerfile in the docker image.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399172203
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -348,18 +349,43 @@ In case of disk space errors on macOS, increase the disk space available for Doc
 Building the Images
 -------------------
 
-You can manually trigger building the local images using the script:
+You can manually trigger building the local CI image using this command:
 
 .. code-block::
 
-  ./breeze build-only
+  ./breeze build-ci-image
 
-The scripts that build the images are optimized to minimize the time needed to rebuild the image when
+And production image using this command:
+
+.. code-block::
+
+  ./breeze build-prod-image
+
+
+The images are build with default extras - different for CI image and different for production image
+and using python version chosen in breeze. You can change those parameters by specifying them as
+command line options. You can see default extras used via  ``./breeze flags`` or in the syntax example
+below.
+
+For example if you want to build python 3.7 version of production image with
+all extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-prod-image --python 3.7 --extras "all"
 
 Review comment:
   I think it's not worth explaining it. 

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


With regards,
Apache Git Services

[GitHub] [airflow] xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396970218
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+# Note that we have to hard-code user id/group id as you cannot use
+# args nor variables in --chown :(. That's a limitation of
+# docker build.
+COPY --chown="50000:50000" --from=airflow-build-image \
+        /root/.local "/home/${AIRFLOW_USER}/.local"
 
 Review comment:
   @potiuk 
   I can build the image with
   ```
   COPY --chown=${AIRFLOW_GID}:${AIRFLOW_UID} --from=airflow-build-image \
           /root/.local "/home/${AIRFLOW_USER}/.local"
   ```
   
   Is it something to do with the docker engine version? My version is 
   
   ```
   Client: Docker Engine - Community
    Version:           19.03.6
    API version:       1.40
    Go version:        go1.12.16
    Git commit:        369ce74a3c
    Built:             Thu Feb 13 01:27:49 2020
    OS/Arch:           linux/amd64
    Experimental:      false
   
   Server: Docker Engine - Community
    Engine:
     Version:          19.03.6
     API version:      1.40 (minimum version 1.12)
     Go version:       go1.12.16
     Git commit:       369ce74a3c
     Built:            Thu Feb 13 01:26:21 2020
     OS/Arch:          linux/amd64
     Experimental:     false
    containerd:
     Version:          1.2.10
     GitCommit:        b34a5c8af56e510852c35414db4c1f4fa6172339
    runc:
     Version:          1.0.0-rc8+dev
     GitCommit:        3e425f80a8c931f88e6d94a8c831b9d5aa481657
    docker-init:
     Version:          0.18.0
     GitCommit:        fec3683
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400489280
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
+
+Those variables should be overwritten only if the production image is built from local sources
+rather than from GitHub repository.
+
+* COPY_SOURCE - should be set to "." if you the image is built from sources
+* COPY_TARGET - should be set to a airflow source directory inside container if the image is built
+  from sources
+* AIRFLOW_SOURCES - should point to the same as COPY_TARGET in case production image is installed from
+  the local sources rather than from Github repository. If left empty it points to sources from
+  GitHub repository derived from AIRFLOW_ORG, AIRFLOW_REPO, AIRFLOW_GIT_REFERENCE
+* CONSTRAINT_REQUIREMENTS should point to locally available requirements file if the image is built
+  from sources. If left empty it will point to the right requirement file from GitHub repository
+  derived from AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+* ENTRYPOINT_FILE - sources point to entrypoint.sh file when installing from sources if left empty
+  it points to sources from GitHub repository derived from
+  AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+
+The below will production image in version 3.6 with default extras from master branch in Github:
+
+.. code-block::
+
+  docker build .
+
+The below will build the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github. Note that versions 1.10.9 and below
+have no requirements so requirements should be taken from head of the v1-10-test branch. Once we
+release 1.10.10 we can take them from the 1.10.10 tag.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg AIRFLOW_GIT_REFERENCE=1.10.9 \
+    --build-arg REQUIREMENTS_GIT_REFERENCE=v1-10-test --build-arg WWW_FOLDER="www_rbac"
+
+The below will build the production image in version 3.7 with default extras from master branch in Github.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+The below will build the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
 
 Review comment:
   Thanks!

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400938695
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
 
 Review comment:
   Oh! Fantastic! Thanks! I was just about to convert them :)
   

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399268830
 
 

 ##########
 File path: .travis.yml
 ##########
 @@ -41,24 +41,24 @@ jobs:
       stage: pre-test
       script: ./scripts/ci/ci_run_all_static_checks.sh
       env: >-
-        PYTHON_VERSION=3.6
+        PYTHON_MAJOR_MINOR_VERSION=3.6
 
 Review comment:
   Thanks!

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401578479
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -263,8 +310,8 @@ to enter the running container. It's as easy as launching ``breeze exec`` while
 Breeze environment. You will be dropped into bash and environment variables will be read in the same
 way as when you enter the environment. You can do it multiple times and open as many terminals as you need.
 
-Stopping Breeze
----------------
+Stopping Interactive environment
 
 Review comment:
   There is a reference to this section where you need to change it, too. see https://github.com/apache/airflow/blob/master/BREEZE.rst#stopping-the-environment

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399269486
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -348,18 +349,43 @@ In case of disk space errors on macOS, increase the disk space available for Doc
 Building the Images
 -------------------
 
-You can manually trigger building the local images using the script:
+You can manually trigger building the local CI image using this command:
 
 .. code-block::
 
-  ./breeze build-only
+  ./breeze build-ci-image
 
-The scripts that build the images are optimized to minimize the time needed to rebuild the image when
+And production image using this command:
+
+.. code-block::
+
+  ./breeze build-prod-image
+
+
+The images are build with default extras - different for CI image and different for production image
+and using python version chosen in breeze. You can change those parameters by specifying them as
+command line options. You can see default extras used via  ``./breeze flags`` or in the syntax example
+below.
+
+For example if you want to build python 3.7 version of production image with
+all extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-prod-image --python 3.7 --extras "all"
 
 Review comment:
   Yeah, I think it's worth adding a note. Not everyone is using breeze and we should not make breeze a requirement for building a docker image :)

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400870300
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
 
 Review comment:
   ```suggestion
   In Breeze by default, the images are built using local sources of Apache Airflow. However
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401511888
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
 
 Review comment:
   Oh `docker pull apache-airflow` would be fine, yeah.
   
   How quickly can we iterate on our Dockerfile if needed once it makes it in to the "library"

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399842790
 
 

 ##########
 File path: .travis.yml
 ##########
 @@ -148,6 +148,20 @@ jobs:
         INSTALL_AIRFLOW_VERSION="1.10.9"
       stage: test
       script: ./scripts/ci/ci_prepare_and_test_backport_packages.sh
+    - name: "Build production image Py3.6"
+      env: >-
+        PYTHON_MAJOR_MINOR_VERSION="3.6"
+      stage: test
+      script: ./scripts/ci/ci_build_production_image.sh
+      before_install:
+        - echo
+    - name: "Build production image Py3.7"
+      env: >-
+        PYTHON_MAJOR_MINOR_VERSION="3.7"
+      stage: test
+      script: ./scripts/ci/ci_build_production_image.sh
+      before_install:
+        - echo
 
 Review comment:
   Yes. The old rule is that the only way you can test if something works is to run it often. I believe in running everything we want to keep working via CI environment. This run will be rather fast usually (2-3 minutes) as it will be using cache once we push the image to dockerhub.

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


With regards,
Apache Git Services

[GitHub] [airflow] martin-kokos commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
martin-kokos commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400970111
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
 
 Review comment:
   ```
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | Environment variable                     | Default value              | Description                                                                            |
   +==========================================+============================+========================================================================================+
   | ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster`` | Base python image                                                                      |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``             | version of Airflow                                                                     |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                    | major/minor version of Python (should match base image)                                |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                      | increasing this number will reinstall all apt dependencies                             |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``KUBECTL_VERSION``                      | ``v1.15.3``                | version of kubectl installed                                                           |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``KIND_VERSION``                         | ``v0.6.1``                 | version of kind installed                                                              |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``PIP_NO_CACHE_DIR``                     | ``true``                   | if true, then no pip cache will be stored                                              |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``PIP_VERSION``                          | ``19.0.2``                 | version of PIP to use                                                                  |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``HOME``                                 | ``/root``                  | Home directory of the root user (CI image has root user as default)                    |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``AIRFLOW_HOME``                         | ``/root/airflow``          | Airflow's HOME (that's where logs and sqlite databases are stored)                     |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``AIRFLOW_SOURCES``                      | ``/opt/airflow``           | Mounted sources of Airflow                                                             |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                      | increasing that number will reinstall all PIP dependencies                             |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``CASS_DRIVER_NO_CYTHON``                | ``1``                      | if set to 1 no CYTHON compilation is done for cassandra driver (much faster)           |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                   | if set then PIP dependencies are installed from repo first before they are reinstalled |
   |                                          |                            | from local sources. This allows for incremental faster builds when requirements change |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``AIRFLOW_REPO``                         | ``apache/airflow``         | the repository from which PIP dependencies are installed (CI optimised)                |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``AIRFLOW_BRANCH``                       | ``master``                 | the branch from which PIP dependencies are installed (CI optimised)                    |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                      | increasing this value will reinstall PIP dependencies from the repository from scratch |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``AIRFLOW_EXTRAS``                       | ``all``                    | extras to install                                                                      |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   | ``ADDITIONAL_PYTHON_DEPS``               |                            | additional python dependencies to install                                              |
   +------------------------------------------+----------------------------+----------------------------------------------------------------------------------------+
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401195449
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
 
 Review comment:
   > This way the Dockerfile becomes a "standalone" Dockerfile.
   >
   > The idea is that once we release Airflow 1.10.10 you should be able to just take the Dockerfile (and nothing else) put it in an empty folder and run:
   
   This is different to every other project I've ever seen build a docker file. Why do we want this?
   
   Honestly if I was wanting to build an image for version 1.10.10, I'd expect to check out that branch/tag and do `docker build .` or a close variation of it.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402462210
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,332 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN if [[ -f "${AIRFLOW_WWW}/package.json" ]]; then \
+        yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache; \
+        yarn --cwd ${AIRFLOW_WWW} run prod; \
+        rm -rf ${AIRFLOW_WWW}/node_modules; \
+    fi
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi6 \
+           libsasl2-2 \
+           libsasl2-modules \
+           libssl1.1 \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           rsync \
+           sasl2-bin \
+           sqlite3 \
+           sudo \
+           unixodbc \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient20 \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+RUN pip install --upgrade pip==${PIP_VERSION}
 
 Review comment:
   The problem is that they need different slightly different prerequisites. If you look closer build image has more -dev dependencies and "build-essentials" which are needed to build some of the extras - but the main image does not have it. similarly libmysqlclient-dev is needed for build but libmysqlient20 is used for production one (no development dependencies needed) ,  This makes main image much smaller.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396456492
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+COPY --chown="50000:50000" --from=airflow-build-image \
 
 Review comment:
   That does not work unfortunately. ARGS do not work as shown parameters in Docker

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402338613
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   This is really a trade-off between the size of the image and download bandwidth/time.
   
   By deleting the .pyc files we get around 100 MB less in the layer. Those are binary files, so they do not compress well. I can get the exact numbers to see but I thought the size difference is quite significant.
   
   I think it's milliseconds of .pyc generation (and it's also distributed over time) vs. downloading 100MB more or so. 
   
   I can get the numbers in a moment so that we can make a better decision on that - both timing of pyc/no-pyc generation for typical airflow startup vs. size of the image. 
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402328081
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   It's not _quite_ the same, but this is almost like we are copying the .java files over and purposefully leaving the `.class` files out, to then recompile them later.
   
   (Yes, it's just a cache in Python so this isn't quite the same I know).
   
   Still, it seems like we are putting extra work in to every container that ever starts up :) 

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


With regards,
Apache Git Services

[GitHub] [airflow] dimberman edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
dimberman edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607974554
 
 
   @potiuk I can't tell if I need to reload something but I'm not seeing build-image in auto-complete
   
    ```
   (airflow) dimberman airflow ((HEAD detached at polidea/add-production-docker-image)) (⎈ |N/A|N/A)
    $ ./breeze build-
   build-docs  build-only
   (airflow) dimberman airflow ((HEAD detached at polidea/add-production-docker-image)) (⎈ |N/A|N/A)
    $ ./breeze build-image --production-image
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402407855
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   WDYT @ashb @mik-laj @kaxil @turbaszek @feluelle -> should we leave or remove .pyc files ? It's a difficult decision to make :)

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402463516
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,332 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN if [[ -f "${AIRFLOW_WWW}/package.json" ]]; then \
+        yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache; \
+        yarn --cwd ${AIRFLOW_WWW} run prod; \
+        rm -rf ${AIRFLOW_WWW}/node_modules; \
+    fi
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi6 \
+           libsasl2-2 \
+           libsasl2-modules \
+           libssl1.1 \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           rsync \
+           sasl2-bin \
+           sqlite3 \
+           sudo \
+           unixodbc \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient20 \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+RUN pip install --upgrade pip==${PIP_VERSION}
 
 Review comment:
   And also when running `breeze build-image --production-image` we are using --cache-from efficiently which makes only sources to rebuild - all the other is taken from the already pulled images.

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402497525
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
 
 Review comment:
   ```suggestion
     docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.6-slim-buster" \
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400883397
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
 
 Review comment:
   ```suggestion
   * Production image (Dockerfile) - used to run production-ready Airflow installations
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401208246
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            DETECTED_DB_USER=${BASH_REMATCH[2]} &&
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            DETECTED_DB_SCHEMA=${BASH_REMATCH[5]} &&
+            DETECTED_DB_PARAMS=${BASH_REMATCH[6]}
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW__CORE__SQL_ALCHEMY_CONN} == sqlite* ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|webserver)$ ]] || \
+        [[ ${DB_RESET:="false"} == "true" ]]; then
+    if [[ ${AIRFLOW_VERSION} == *1.10* ]]; then
+        airflow reset_db -y >/dev/null 2>&1
 
 Review comment:
   Woah. This shouldn't even be an option in a production image, it seems wildly dangerous.

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


With regards,
Apache Git Services

[GitHub] [airflow] martin-kokos commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
martin-kokos commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400944375
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
 
 Review comment:
   Sorry. I can update the comment in a bit.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402507696
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
 
 Review comment:
   Yeah. I am back. Happy to review those :)

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396431539
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
 
 Review comment:
   This direcotories already exists. Look at 90 line. 

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/7939758ee3d9300e1f29bcf38a6830799fca0c85&el=desc) will **decrease** coverage by `0.51%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.16%   86.65%   -0.52%     
   ==========================================
     Files         931      931              
     Lines       45172    45361     +189     
   ==========================================
   - Hits        39375    39306      -69     
   - Misses       5797     6055     +258     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [4 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [7939758...ce9c4d1](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-606709575
 
 
   All corrections made! 

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402324811
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,333 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
 
 Review comment:
   Still seems odd to me that this is an argument - I can't see when we'd chagne it?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402339488
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,333 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
 
 Review comment:
   Ah right indeed. I forgot that we have it pre-compiled in released packages. Good eye. Thanks!

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399158527
 
 

 ##########
 File path: .travis.yml
 ##########
 @@ -41,24 +41,24 @@ jobs:
       stage: pre-test
       script: ./scripts/ci/ci_run_all_static_checks.sh
       env: >-
-        PYTHON_VERSION=3.6
+        PYTHON_MAJOR_MINOR_VERSION=3.6
 
 Review comment:
   Why do we need to change it? I don't understand what means "major_minor" :<  

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402517891
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
 
 Review comment:
   Solved/.

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607977704
 
 
   @dimberman run `breeze-complete` script again.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402321490
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   > and the cache will be generated on the first run of python interpreter for the main image
   
   I'd probably suggest that we have it written here -- that way the production image doesn't have to make any changes (it's often not possible, but it's nice to be able to run docker containers with a ro root disk).
   
   I.e. it's nice to be able to run `docker diff airflow` and not see a bunch of .pyc files scattered about. 

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401205424
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_SOURCES}[${AIRFLOW_EXTRAS}]" \
+    --constraint /requirements.txt
+
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
 
 Review comment:
   This one seems somewhat redundant.
   
   ```suggestion
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396457700
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+COPY --chown="50000:50000" --from=airflow-build-image \
 
 Review comment:
   Added comment about it.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401828047
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -263,8 +310,8 @@ to enter the running container. It's as easy as launching ``breeze exec`` while
 Breeze environment. You will be dropped into bash and environment variables will be read in the same
 way as when you enter the environment. You can do it multiple times and open as many terminals as you need.
 
-Stopping Breeze
----------------
+Stopping Interactive environment
 
 Review comment:
   I removed the reference. It was completely unnecessary :)

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


With regards,
Apache Git Services

[GitHub] [airflow] martin-kokos commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
martin-kokos commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400972079
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
 
 Review comment:
   ```
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | Argument                          | Example value                                  | Description                                                                                                                         |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``PYTHON_BASE_IMAGE``             | ``python:3.6-slim-buster``                     | Base python image                                                                                                                   |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``PYTHON_MAJOR_MINOR_VERSION``    | ``3.6``                                        | major/minor version of Python (should match base image)                                                                             |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``AIRFLOW_VERSION``               | ``2.0.0.dev0``                                 | version of Airflow                                                                                                                  |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``AIRFLOW_ORG``                   | ``apache``                                     | Github organisation from which Airflow is installed (when installed from repo)                                                      |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``AIRFLOW_REPO``                  | ``airflow``                                    | Github repository from which Airflow is installed (when installed from repo)                                                        |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``AIRFLOW_GIT_REFERENCE``         | ``master``                                     | reference (branch or tag) from Github repository from which Airflow is installed (when installed from repo)                         |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``REQUIREMENTS_GIT_REFERENCE``    | ``master``                                     | reference (branch or tag) from Github repository from which requirements are downloaded for constraints (when installed from repo). |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``WWW_FOLDER``                    | ``www`` - folder where www pages are generated | it should be set to www_rbac in case of 1.10 image builds.                                                                          |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``AIRFLOW_EXTRAS``                | (see Dockerfile)                               | Default extras with which airflow is installed                                                                                      |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``AIRFLOW_HOME``                  | ``/opt/airflow``                               | Airflow's HOME (that's where logs and sqlite databases are stored)                                                                  |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``AIRFLOW_UID``                   | ``50000``                                      | Airflow user UID                                                                                                                    |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``AIRFLOW_GID``                   | ``50000``                                      | Airflow group GID                                                                                                                   |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``PIP_VERSION``                   | ``19.0.2``                                     | version of PIP to use                                                                                                               |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   | ``CASS_DRIVER_BUILD_CONCURRENCY`` | ``8``                                          | Number of processors to use for cassandra PIP install (speeds up installing in case cassandra extra is used).                       |
   +-----------------------------------+------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------+
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402516864
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
 
 Review comment:
   Ahh, missed that one :). Feel free to PR :)

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/51be6ec1c0f430d964c3f32895a9f909d3ff8300&el=desc) will **decrease** coverage by `0.38%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.16%   86.78%   -0.39%     
   ==========================================
     Files         932      932              
     Lines       45187    45191       +4     
   ==========================================
   - Hits        39389    39220     -169     
   - Misses       5798     5971     +173     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | [airflow/providers/mongo/hooks/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vaG9va3MvbW9uZ28ucHk=) | `93.24% <0.00%> (-4.06%)` | :arrow_down: |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <0.00%> (-0.89%)` | :arrow_down: |
   | ... and [1 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [51be6ec...81e6c3a](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-606860520
 
 
   I don't see why we want to build from remote sources at all. I can't see where it would be used.
   
   Where does the Dockerfile come from to do such a build? If we got it out of the branch/tag for the repo: why not just build form the checkout rather than having to go fetch it again?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399168374
 
 

 ##########
 File path: .travis.yml
 ##########
 @@ -41,24 +41,24 @@ jobs:
       stage: pre-test
       script: ./scripts/ci/ci_run_all_static_checks.sh
       env: >-
-        PYTHON_VERSION=3.6
+        PYTHON_MAJOR_MINOR_VERSION=3.6
 
 Review comment:
   I already explained it before - because  PYTHON_VERSION is set to 3.6.9 in the base Python images and it causes a lot of confusion as depending on the script sometimes you can get PYTHON_VERSION = 3.6.9. 
   
   MAJOR_MINOR_VERSION is pretty straightforward IMHO. Do you have any other proposal. The requirements for the name are:
   
   1) can't be PYTHON_VERSION (already set by the base image)
   2) Has to correctly reflect 3.9 being MAJOR.MINOR version of python
   3) Cannot have "." (environment variable can't have dots).
   
   What is your proposal for the name of this variable please ?
   
   
   
   

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400872162
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
 
 Review comment:
   ![image](https://user-images.githubusercontent.com/8811558/78026361-6dcf3f80-7353-11ea-82b2-ace7c5795970.png)
   
   
   Same here

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401511860
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            DETECTED_DB_USER=${BASH_REMATCH[2]} &&
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            DETECTED_DB_SCHEMA=${BASH_REMATCH[5]} &&
+            DETECTED_DB_PARAMS=${BASH_REMATCH[6]}
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW__CORE__SQL_ALCHEMY_CONN} == sqlite* ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|webserver)$ ]] || \
+        [[ ${DB_RESET:="false"} == "true" ]]; then
+    if [[ ${AIRFLOW_VERSION} == *1.10* ]]; then
+        airflow reset_db -y >/dev/null 2>&1
 
 Review comment:
   Indeed. I will make it works in Breeze outside of the image/entrypoint.sh itself.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399168374
 
 

 ##########
 File path: .travis.yml
 ##########
 @@ -41,24 +41,24 @@ jobs:
       stage: pre-test
       script: ./scripts/ci/ci_run_all_static_checks.sh
       env: >-
-        PYTHON_VERSION=3.6
+        PYTHON_MAJOR_MINOR_VERSION=3.6
 
 Review comment:
   I already explained it before - because  PYTHON_VERSION is set to 3.6.9 in the base Python images and it causes a lot of confusion as depending on the script sometimes you can get PYTHON_VERSION = 3.6.9 and sometimes 3.6. 
   
   MAJOR_MINOR_VERSION is pretty straightforward IMHO. Do you have any other proposal. The requirements for the name are:
   
   1) can't be PYTHON_VERSION (already set by the base image)
   2) Has to correctly reflect 3.9 being MAJOR.MINOR version of python
   3) Cannot have "." (environment variable can't have dots).
   
   What is your proposal for the name of this variable please ?
   
   
   
   

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


With regards,
Apache Git Services

[GitHub] [airflow] martin-kokos commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
martin-kokos commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400914471
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
 
 Review comment:
   ```
   Argument | Example value | Description
   -- | -- | --
   ``PYTHON_BASE_IMAGE`` | ``python:3.6-slim-buster`` | Base python image
   ``PYTHON_MAJOR_MINOR_VERSION`` | ``3.6`` | major/minor version of Python (should match base image)
   ``AIRFLOW_VERSION`` | ``2.0.0.dev0`` | version of Airflow
   ``AIRFLOW_ORG`` | ``apache`` | Github organisation from which Airflow is installed (when installed from repo)
   ``AIRFLOW_REPO`` | ``airflow`` | Github repository from which Airflow is installed (when installed from repo)
   ``AIRFLOW_GIT_REFERENCE`` | ``master`` | reference (branch or tag) from Github repository from which Airflow is installed (when installed from repo)
   ``REQUIREMENTS_GIT_REFERENCE`` | ``master`` | reference (branch or tag) from Github repository from which requirements are downloaded for constraints (when installed from repo).
   ``WWW_FOLDER`` | ``www`` - folder where www pages are generated | it should be set to www_rbac in case of 1.10 image builds.
   ``AIRFLOW_EXTRAS`` | (see Dockerfile) | Default extras with which airflow is installed
   ``AIRFLOW_HOME`` | ``/opt/airflow`` | Airflow's HOME (that's where logs and sqlite databases are stored)
   ``AIRFLOW_UID`` | ``50000`` | Airflow user UID
   ``AIRFLOW_GID`` | ``50000`` | Airflow group GID
   ``PIP_VERSION`` | ``19.0.2`` | version of PIP to use
   ``CASS_DRIVER_BUILD_CONCURRENCY`` | ``8`` | Number of processors to use for cassandra PIP install (speeds up installing in case cassandra extra is used).
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400891360
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
+
+Those variables should be overwritten only if the production image is built from local sources
+rather than from GitHub repository.
+
+* COPY_SOURCE - should be set to "." if you the image is built from sources
+* COPY_TARGET - should be set to a airflow source directory inside container if the image is built
+  from sources
+* AIRFLOW_SOURCES - should point to the same as COPY_TARGET in case production image is installed from
+  the local sources rather than from Github repository. If left empty it points to sources from
+  GitHub repository derived from AIRFLOW_ORG, AIRFLOW_REPO, AIRFLOW_GIT_REFERENCE
+* CONSTRAINT_REQUIREMENTS should point to locally available requirements file if the image is built
+  from sources. If left empty it will point to the right requirement file from GitHub repository
+  derived from AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+* ENTRYPOINT_FILE - sources point to entrypoint.sh file when installing from sources if left empty
+  it points to sources from GitHub repository derived from
+  AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+
+The below will production image in version 3.6 with default extras from master branch in Github:
 
 Review comment:
   ```suggestion
   The below will build production image in version 3.6 with default extras from master branch in Github:
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400890454
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
 
 Review comment:
   ```suggestion
   The following arguments can be used for Production images:
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.49%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   86.67%   -0.50%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39168     -223     
   - Misses       5796     6019     +223     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [4 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401826532
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            DETECTED_DB_USER=${BASH_REMATCH[2]} &&
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            DETECTED_DB_SCHEMA=${BASH_REMATCH[5]} &&
+            DETECTED_DB_PARAMS=${BASH_REMATCH[6]}
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW__CORE__SQL_ALCHEMY_CONN} == sqlite* ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|webserver)$ ]] || \
+        [[ ${DB_RESET:="false"} == "true" ]]; then
+    if [[ ${AIRFLOW_VERSION} == *1.10* ]]; then
+        airflow reset_db -y >/dev/null 2>&1
 
 Review comment:
   On the other hand. Indeed it is wildly dangerous. I remove it completely. It's a nuance.

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402502180
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
 
 Review comment:
   ```suggestion
       --build-arg AIRFLOW_SOURCES_FROM="Dockerfile" \
       --build-arg AIRFLOW_SOURCES_TO="/Dockerfile" \
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on issue #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on issue #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-602684843
 
 
   There is some error in the scripts.
   When I ran following command:
   ```bash
   bash breeze restart -a 1.10.6 && bash breeze -a 1.10.6
   ```
   I got following error:
   ```
   Installs version of Airflow: 1.10.6
   
   Restarts the environment. Includes emptying the databases.
   
   Fixing group permissions
   Fixed group permissions
   
   Docker image build is forced for CI image
   
   Checking if the remote image needs to be pulled
   
   /Users/kamilbregula/devel/google-airflow/airflow/scripts/ci/_utils.sh: line 628: AIRFLOW_REMOTE_MANIFEST_IMAGE: unbound variable
   ```
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400504559
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_SOURCES}[${AIRFLOW_EXTRAS}]" \
+    --constraint /requirements.txt
+
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi6 \
+           libpq5 \
 
 Review comment:
   Do we need libpq5 and postgresql-client? I'd have thought that libpq is a dep of the other.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396702435
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
 
 Review comment:
   I am gpoing to backport to 1.10.10
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400628507
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
 
 Review comment:
   Not really. We have two ways of  building the image:
   
   1) From local sources (and then we can use entrypoint.sh locally)
   2) From remote Github URL. And then we need to make sure that the entrypoint.sh file is the same as the version we are installing the image from. We will be able to use the same Dockerfile to install airflow from both 1.10 and 2.0 and there will be difference between those two entryopoint.sh files ('airflow resetdb` vs `airflow db reset` is one that will be there for sure). So we need to make sure to use the correct one.
   
   I want this Dockerfile to become the source of the "official" DockerHub image (https://docs.docker.com/docker-hub/official_images/). And if I do the above - get everything from released versions in GitHub for the "remote" version of building the image =  then we only need the Dockerfile itself and no additional files to become the "official" docker image. 
   
   This way the Dockerfile becomes a "standalone" Dockerfile.
   
   The idea is that once we release Airflow 1.10.10 you should be able to just take the Dockerfile (and nothing else) put it in an empty folder and run:
   
   ``` 
   docker build . --build-arg=AIRFLOW_GIT_REFERENCE=1.10.10 --build-arg=AIRFLOW_VERSION=1.10.10
   ```
   
   and image of Airflow 1.10.10 should be built in a repeatable fashion. 
   
   I managed to achieve exactly this property by adding requirements.txt and downloading everything from github.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.49%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   86.67%   -0.50%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39168     -223     
   - Misses       5796     6019     +223     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [4 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.26%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   86.79%   +0.26%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39353     +146     
   + Misses       6105     5989     -116     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [31 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...602a289](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402415934
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,430 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+Yyou can also build production images from installed packages
+via providing ``--install-airflow-version`` parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earliier version -  both constraint and requirements should point to 1.10.10 version.
+
+Yyou can also build production images from specific Git verion
+via providing ``--install-airflow-reference`` parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from https://raw.githubusercontent.com/apache/airflow/<tag>>/entrypoint.sh
+url. It is important so that we have matching version of the requirements. Similar comment applies
+to 1.10.10 version as in case of v
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
+    --build-arg WWW_FOLDER="www_rbac"
+
+This builds the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE=entrypoint.sh \
+    --build-arg AIRFLOW_INSTALL_SOURCES="apache-airflow" \
+    --build-arg AIRFLOW_INSTALL_VERSION="==1.10.10" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt"
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
+    --build-arg WWW_FOLDER="www_rbac"
+
+Image manifests
+---------------
+
+Together with the main CI images we also build and push image manifests. Those manifests are very small images
+that contain only results of the docker inspect for the image. This is in order to be able to
+determine very quickly if the image in the docker registry has changed a lot since the last time.
+Unfortunately docker registry (specifically dockerhub registry) has no anonymous way of querying image
+details via API, you need to download the image to inspect it. We overcame it in the way that
+always when we build the image we build a very small image manifest and push it to registry together
+with the main CI image. The tag for the manifest image is the same as for the image it refers
+to with added ``-manifest`` suffix. The manifest image for ``apache/airflow:master-python3.6-ci`` is named
+``apache/airflow:master-python3.6-ci-manifest``.
+
+
+Pulling the Latest Images
+-------------------------
+
+Sometimes the image needs to be rebuilt from scratch. This is required, for example,
+when there is a security update of the Python version that all the images are based on and new version
+of the image is pushed to the repository. In this case it is usually faster to pull the latest
+images rather than rebuild them from scratch.
+
+You can do it via the ``--force-pull-images`` flag to force pulling the latest images from the Docker Hub.
+
+For production image:
+
+.. code-block::
+
+  ./breeze build-image --force-pull-images --production-image
+
+For CI image Breeze automatically uses force pulling in case it determines that your image is very outdated,
+however uou can also force it with the same flag.
+
+.. code-block::
+
+  ./breeze build-image --force-pull-images
+
+Using the images
+================
+
+Both images have entrypoint set as dumb-init with entrypoint.sh script executed (in order to forward
+signals). This entrypoint works as follows:
+
+* If ``AIRFLOW__CORE__SQL_ALCHEMY_CONN`` variable is passed to the container and it is either mysql or postgres
+  SQL alchemy connection, then the connection is checked and the script waits until the database is reachable.
+* If no ``AIRFLOW__CORE__SQL_ALCHEMY_CONN`` variable is set or if it is set to sqlite SQL alchemy connection
+  then db reset is executed.
+* If ``AIRFLOW__CELERY__BROKER_URL`` variable is passed and scheduler, worker of flower command is used then
+  the connection is checked and the script waits until the Celery broker database is reachable.
+* If ``DB_RESET`` is specified - the 'airflow db reset' is used.
+
+* If no argument is specified - you are dropped in bash shell.
+* If there are any arguments they are passed to "airflow" command
 
 Review comment:
   I think more info and docs will come when we have helm chart. I see the Dockerfile as a starting point and it's the helm chart where we should see more use cases explained.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402322231
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   I see later in the file you are explicitly deleting them -- did this env var not actually do anything anyway?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401511860
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            DETECTED_DB_USER=${BASH_REMATCH[2]} &&
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            DETECTED_DB_SCHEMA=${BASH_REMATCH[5]} &&
+            DETECTED_DB_PARAMS=${BASH_REMATCH[6]}
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW__CORE__SQL_ALCHEMY_CONN} == sqlite* ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|webserver)$ ]] || \
+        [[ ${DB_RESET:="false"} == "true" ]]; then
+    if [[ ${AIRFLOW_VERSION} == *1.10* ]]; then
+        airflow reset_db -y >/dev/null 2>&1
 
 Review comment:
   Indeed. I will make it works in Breeze outside of the image itself.

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


With regards,
Apache Git Services

[GitHub] [airflow] martin-kokos commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
martin-kokos commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400909834
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
 
 Review comment:
   Here is table. Hope it helps
   ```
   Environment variable | Default value | Description
   --|--|--
   ``PYTHON_BASE_IMAGE`` | ``python:3.6-slim-buster`` | Base python image
   ``AIRFLOW_VERSION`` | ``2.0.0.dev0`` | version of Airflow
   ``PYTHON_MAJOR_MINOR_VERSION`` | ``3.6`` | major/minor version of Python (should match base image)
   ``DEPENDENCIES_EPOCH_NUMBER`` | ``2`` | increasing this number will reinstall all apt dependencies
   ``KUBECTL_VERSION`` | ``v1.15.3`` | version of kubectl installed
   ``KIND_VERSION`` | ``v0.6.1`` | version of kind installed
   ``PIP_NO_CACHE_DIR`` | ``true`` | if true, then no pip cache will be stored
   ``PIP_VERSION`` | ``19.0.2`` | version of PIP to use
   ``HOME`` | ``/root`` | Home directory of the root user (CI image has root user as default)
   ``AIRFLOW_HOME`` | ``/root/airflow`` | Airflow's HOME (that's where logs and sqlite databases are stored)
   ``AIRFLOW_SOURCES`` | ``/opt/airflow`` | Mounted sources of Airflow
   ``PIP_DEPENDENCIES_EPOCH_NUMBER`` | ``3`` | increasing that number will reinstall all PIP dependencies
   ``CASS_DRIVER_NO_CYTHON`` | ``1`` | if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
   ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true`` if set then PIP dependencies are installed from repo first before they are reinstalled from local sources. This allows for incremental faster builds when requirements change
   ``AIRFLOW_REPO`` | ``apache/airflow`` | the repository from which PIP dependencies are installed (CI optimised)
   ``AIRFLOW_BRANCH`` | ``master`` | the branch from which PIP dependencies are installed (CI optimised)
   ``AIRFLOW_CI_BUILD_EPOCH`` | ``1`` | increasing this value will reinstall PIP dependencies from the repository from scratch
   ``AIRFLOW_EXTRAS`` | ``all`` | extras to install
   ``ADDITIONAL_PYTHON_DEPS`` | ```` | additional python dependencies to install
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396433481
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   Should we remove this variable after installation? In production, we wanna use a bytecode to speed-up worker.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400502435
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
 
 Review comment:
   I'd really rather we didn't have nodejs in our production images.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402413029
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   Okay yeah, that's big enough a difference to say leave it out.
   
   For 3.8+ it might be worth looking at https://docs.python.org/3/using/cmdline.html#envvar-PYTHONPYCACHEPREFIX to make it store the PYC files elsewhere in the tree (i.e. in to a `/tmp` somewhere, both at build image and prod image runtime)

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400624728
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
 
 Review comment:
   See IMAGES.rst. It's all explained there. I am using the same Dockerfile to build from a) sources or b) from remote URL. The construct that I am using here allows to specify different ways how we get the sources and build them using the same way during the CI (so that we can test the production Docker image from sources) or from the released github version  of ta airflow. This way we can get stable, repeatable release from Github Sources as well as continuous testing of the image during our CI builds.
   
   We are going to use this very image (build from sources) and helm charts donated by Astronomer to run tests of all three together: Dockerfile (image), helm chart and tests of various executors installed with the helm charts. 

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401823428
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
 
 Review comment:
   Thanks for pointing out @ashb !

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402397386
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
 
 Review comment:
   I agree with @kaxil change

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396443619
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
 
 Review comment:
   Not sure. I will do some round of optimisations and see what else I can remove, but it will require some testing.

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400872518
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
+
+Those variables should be overwritten only if the production image is built from local sources
+rather than from GitHub repository.
+
+* COPY_SOURCE - should be set to "." if you the image is built from sources
+* COPY_TARGET - should be set to a airflow source directory inside container if the image is built
+  from sources
+* AIRFLOW_SOURCES - should point to the same as COPY_TARGET in case production image is installed from
+  the local sources rather than from Github repository. If left empty it points to sources from
+  GitHub repository derived from AIRFLOW_ORG, AIRFLOW_REPO, AIRFLOW_GIT_REFERENCE
+* CONSTRAINT_REQUIREMENTS should point to locally available requirements file if the image is built
+  from sources. If left empty it will point to the right requirement file from GitHub repository
+  derived from AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+* ENTRYPOINT_FILE - sources point to entrypoint.sh file when installing from sources if left empty
+  it points to sources from GitHub repository derived from
+  AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+
+The below will production image in version 3.6 with default extras from master branch in Github:
+
+.. code-block::
+
+  docker build .
+
+The below will build the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github. Note that versions 1.10.9 and below
+have no requirements so requirements should be taken from head of the v1-10-test branch. Once we
+release 1.10.10 we can take them from the 1.10.10 tag.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg AIRFLOW_GIT_REFERENCE=1.10.9 \
+    --build-arg REQUIREMENTS_GIT_REFERENCE=v1-10-test --build-arg WWW_FOLDER="www_rbac"
+
+The below will build the production image in version 3.7 with default extras from master branch in Github.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+The below will build the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE=entrypoint.sh
+
+Image manifests
+---------------
+
+Together with teh main images we also build an push image manifests. Those manifests are vey small images
 
 Review comment:
   ```suggestion
   Together with the main images we also build a push image manifests. Those manifests are very small images
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400506104
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
 
 Review comment:
   Why is this needed? Isn't `COPY entrypoint.sh /entrypoint` the same and easier to reason about?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399181180
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -348,18 +349,43 @@ In case of disk space errors on macOS, increase the disk space available for Doc
 Building the Images
 -------------------
 
-You can manually trigger building the local images using the script:
+You can manually trigger building the local CI image using this command:
 
 .. code-block::
 
-  ./breeze build-only
+  ./breeze build-ci-image
 
-The scripts that build the images are optimized to minimize the time needed to rebuild the image when
+And production image using this command:
+
+.. code-block::
+
+  ./breeze build-prod-image
+
+
+The images are build with default extras - different for CI image and different for production image
+and using python version chosen in breeze. You can change those parameters by specifying them as
+command line options. You can see default extras used via  ``./breeze flags`` or in the syntax example
+below.
+
+For example if you want to build python 3.7 version of production image with
+all extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-prod-image --python 3.7 --extras "all"
 
 Review comment:
   I will actually add it to the docs :)

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.64%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   87.17%   +0.64%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39526     +319     
   + Misses       6105     5816     -289     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [35 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...3b336c6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.49%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   86.67%   -0.50%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39168     -223     
   - Misses       5796     6019     +223     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [4 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396946424
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
 
 Review comment:
   Yes. See the comment above just before the second "FROM". 
   
   We have two images here" 
   * The "build" image is there to run "pip install" - it needs MySQL to install the MySQL dev library and pip dependencies. But we only use that image to run the "pip install --user" and store the dependencies and libraries compiled during install in "${HOME}/.local"
   * The actual Airflow image: this one has no "build" dependencies (build-essentials) and during optimization we can remove few more unneeded libraries. Then from the "build" image we ONLY take the ".local" directory (this contains all python pip-insttaled dependencies and all binary-compiled .so libraries needed. This way the resulting Airflow image is much smaller. 
   
   We need the MySQL libraries in Airflow image because ... we want to connect to MySQL :).

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396446834
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
 
 Review comment:
   Do we want to install it for every user? This is only needed when extra == all, kerberos, devel or devel_hadopp (I'm not sure of the last one)

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400628507
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
 
 Review comment:
   Not really. We have two ways of  building the image:
   
   1) From local sources (and then we can use entrypoint.sh locally)
   2) From remote Github URL. And then we need to make sure that the entrypoint.sh file is the same as the version we are installing the image from. We will be able to use the same Dockerfile to install airflow from both 1.10 and 2.0 and there will be difference between those two entryopoint.sh files (`airflow resetdb` vs `airflow db reset` is one that will be there for sure). So we need to make sure to use the correct one.
   
   I want this Dockerfile to become the source of the "official" DockerHub image (https://docs.docker.com/docker-hub/official_images/). And if I do the above - get everything from released versions in GitHub for the "remote" version of building the image =  then we only need the Dockerfile itself and no additional files to become the "official" docker image. 
   
   This way the Dockerfile becomes a "standalone" Dockerfile.
   
   The idea is that once we release Airflow 1.10.10 you should be able to just take the Dockerfile (and nothing else) put it in an empty folder and run:
   
   ``` 
   docker build . --build-arg=AIRFLOW_GIT_REFERENCE=1.10.10 --build-arg=AIRFLOW_VERSION=1.10.10
   ```
   
   and image of Airflow 1.10.10 should be built in a repeatable fashion. 
   
   I managed to achieve exactly this property by adding requirements.txt and downloading everything from github.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396456492
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+COPY --chown="50000:50000" --from=airflow-build-image \
 
 Review comment:
   That does not work unfortunately. ARGS do not work as chown parameters in Docker

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400868676
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
 
 Review comment:
   ```suggestion
   * ``PYTHON_MAJOR_MINOR_VERSION`` - version of python used to build the image. Examples: 3.5, 3.7
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607936924
 
 
   :tada: :tada: !

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io commented on issue #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/7939758ee3d9300e1f29bcf38a6830799fca0c85&el=desc) will **decrease** coverage by `0.51%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.16%   86.65%   -0.52%     
   ==========================================
     Files         931      931              
     Lines       45172    45361     +189     
   ==========================================
   - Hits        39375    39306      -69     
   - Misses       5797     6055     +258     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [4 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [7939758...ce9c4d1](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400860574
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
 
 Review comment:
   ```suggestion
   # NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] zhongjiajie commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402460883
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,332 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN if [[ -f "${AIRFLOW_WWW}/package.json" ]]; then \
+        yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache; \
+        yarn --cwd ${AIRFLOW_WWW} run prod; \
+        rm -rf ${AIRFLOW_WWW}/node_modules; \
+    fi
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi6 \
+           libsasl2-2 \
+           libsasl2-modules \
+           libssl1.1 \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           rsync \
+           sasl2-bin \
+           sqlite3 \
+           sudo \
+           unixodbc \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient20 \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+RUN pip install --upgrade pip==${PIP_VERSION}
 
 Review comment:
   > Could we add one more stage, for the same install, just like install/upgrade pip, and install mysql-client?
   
   Also the `apt-get install` part I find out their have the same package install in `airflow-build-image` and `main`

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-602686438
 
 
   > /Users/kamilbregula/devel/google-airflow/airflow/scripts/ci/_utils.sh: line 628: AIRFLOW_REMOTE_MANIFEST_IMAGE: unbound variable
   > ```
   
   will take a look thanks!

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.26%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   86.79%   +0.26%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39353     +146     
   + Misses       6105     5989     -116     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [31 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...602a289](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401530747
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
 
 Review comment:
   (It doesn't matter/affect anything much, and wouldn't have commented unless there were other changes to make.)

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400625279
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_SOURCES}[${AIRFLOW_EXTRAS}]" \
+    --constraint /requirements.txt
+
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi6 \
+           libpq5 \
 
 Review comment:
   I will remove and see if it still works.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil merged pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil merged pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832
 
 
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399788839
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
 
 Review comment:
   I think we should leave it.

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r397065441
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
 
 Review comment:
   Got it!

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400870181
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
 
 Review comment:
   ```suggestion
   The command that builds the production image is optimised for the size of the image.
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400872988
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
 
 Review comment:
   I will convert to tables. I was just lazy  :)

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401208609
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            DETECTED_DB_USER=${BASH_REMATCH[2]} &&
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            DETECTED_DB_SCHEMA=${BASH_REMATCH[5]} &&
+            DETECTED_DB_PARAMS=${BASH_REMATCH[6]}
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW__CORE__SQL_ALCHEMY_CONN} == sqlite* ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|webserver)$ ]] || \
+        [[ ${DB_RESET:="false"} == "true" ]]; then
+    if [[ ${AIRFLOW_VERSION} == *1.10* ]]; then
 
 Review comment:
   This could match 2.1.10 in future

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396454927
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
 
 Review comment:
   That's somethig to be discussed. It's one of the optimizations we might want to consider. That's why I need people to test the image and discuss it on the devlist. Without input from the users it's hard to say if this will be highly requested or not. 
   
   We have to balance the reusability with the size of the image. Sometimes we might choose slightly bigger image but one that will cover more cases.
   
   I think input from users is most important. 

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396434064
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
 
 Review comment:
   They don't. This is multo-segmented image:
   
   FROM ${PYTHON_BASE_IMAGE} as main line above)
   
   The lines before that are for the "build" image that contains all the build essentials and allows to build airflow c-dependencies. After that line there is another image - the actual production one  - which has only basic dependencies (no build-essentials for example) thus is much smaller - and I only copy installed Airflow and all it's dependencies from it.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399171768
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -348,18 +349,43 @@ In case of disk space errors on macOS, increase the disk space available for Doc
 Building the Images
 -------------------
 
-You can manually trigger building the local images using the script:
+You can manually trigger building the local CI image using this command:
 
 .. code-block::
 
-  ./breeze build-only
+  ./breeze build-ci-image
 
-The scripts that build the images are optimized to minimize the time needed to rebuild the image when
+And production image using this command:
+
+.. code-block::
+
+  ./breeze build-prod-image
+
+
+The images are build with default extras - different for CI image and different for production image
+and using python version chosen in breeze. You can change those parameters by specifying them as
+command line options. You can see default extras used via  ``./breeze flags`` or in the syntax example
+below.
+
+For example if you want to build python 3.7 version of production image with
+all extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-prod-image --python 3.7 --extras "all"
 
 Review comment:
   IT will work  in one combination and it is slower because it is multi-segmented image which cannot be cached directly. If you want to reflect the actual build command it should be something like 
   
   ```
   docker build . --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg AIRFLOW_EXTRAS="all" --cache-from apache/airflow:master-build-3.7 --cache-from apache/airflow:master-3.7 
   ```
   

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


With regards,
Apache Git Services

[GitHub] [airflow] zhongjiajie commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402458438
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,332 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN if [[ -f "${AIRFLOW_WWW}/package.json" ]]; then \
+        yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache; \
+        yarn --cwd ${AIRFLOW_WWW} run prod; \
+        rm -rf ${AIRFLOW_WWW}/node_modules; \
+    fi
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi6 \
+           libsasl2-2 \
+           libsasl2-modules \
+           libssl1.1 \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           rsync \
+           sasl2-bin \
+           sqlite3 \
+           sudo \
+           unixodbc \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient20 \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+RUN pip install --upgrade pip==${PIP_VERSION}
 
 Review comment:
   It will be more easy to maintain cause we just need to change one part instead of two, and make the build more faster?

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


With regards,
Apache Git Services

[GitHub] [airflow] xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396834360
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
 
 Review comment:
   Is this necessary for development because it will get swamped by the installation log really quickly? If it is for providing a summary after the build, we can push this down near the `CMD` command

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400488890
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
+
+Those variables should be overwritten only if the production image is built from local sources
+rather than from GitHub repository.
+
+* COPY_SOURCE - should be set to "." if you the image is built from sources
+* COPY_TARGET - should be set to a airflow source directory inside container if the image is built
+  from sources
+* AIRFLOW_SOURCES - should point to the same as COPY_TARGET in case production image is installed from
+  the local sources rather than from Github repository. If left empty it points to sources from
+  GitHub repository derived from AIRFLOW_ORG, AIRFLOW_REPO, AIRFLOW_GIT_REFERENCE
+* CONSTRAINT_REQUIREMENTS should point to locally available requirements file if the image is built
+  from sources. If left empty it will point to the right requirement file from GitHub repository
+  derived from AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+* ENTRYPOINT_FILE - sources point to entrypoint.sh file when installing from sources if left empty
+  it points to sources from GitHub repository derived from
+  AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+
+The below will production image in version 3.6 with default extras from master branch in Github:
+
+.. code-block::
+
+  docker build .
+
+The below will build the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github. Note that versions 1.10.9 and below
+have no requirements so requirements should be taken from head of the v1-10-test branch. Once we
+release 1.10.10 we can take them from the 1.10.10 tag.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg AIRFLOW_GIT_REFERENCE=1.10.9 \
+    --build-arg REQUIREMENTS_GIT_REFERENCE=v1-10-test --build-arg WWW_FOLDER="www_rbac"
+
+The below will build the production image in version 3.7 with default extras from master branch in Github.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+The below will build the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
 
 Review comment:
   ```suggestion
       --build-arg CONSTRAINT_REQUIREMENTS="requirements/requirements-python3.7.txt" \
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] dimberman commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396687635
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
 
 Review comment:
   Is the image going to start at 2.0 or will we backport later?

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396448378
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
 
 Review comment:
   Can you say something more about that? I'm afraid this may not work in Docker.

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


With regards,
Apache Git Services

[GitHub] [airflow] dimberman commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396686998
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -348,18 +349,43 @@ In case of disk space errors on macOS, increase the disk space available for Doc
 Building the Images
 -------------------
 
-You can manually trigger building the local images using the script:
+You can manually trigger building the local CI image using this command:
 
 .. code-block::
 
-  ./breeze build-only
+  ./breeze build-ci-image
 
-The scripts that build the images are optimized to minimize the time needed to rebuild the image when
+And production image using this command:
+
+.. code-block::
+
+  ./breeze build-prod-image
 
 Review comment:
   +1

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402413510
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,333 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
 
 Review comment:
   No no problem with it, just wasn't quite sure what purpose it served.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400639827
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_SOURCES}[${AIRFLOW_EXTRAS}]" \
+    --constraint /requirements.txt
+
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi6 \
+           libpq5 \
 
 Review comment:
   Works :)

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


With regards,
Apache Git Services

[GitHub] [airflow] xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396835351
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+COPY --chown="50000:50000" --from=airflow-build-image \
 
 Review comment:
   I can build the image with
   ```
   COPY --chown=${AIRFLOW_GID}:${AIRFLOW_UID} --from=airflow-build-image \
           /root/.local "/home/${AIRFLOW_USER}/.local"
   ```
   
   Is it something to do with the docker engine version? My version is 
   
   ```
   Client: Docker Engine - Community
    Version:           19.03.6
    API version:       1.40
    Go version:        go1.12.16
    Git commit:        369ce74a3c
    Built:             Thu Feb 13 01:27:49 2020
    OS/Arch:           linux/amd64
    Experimental:      false
   
   Server: Docker Engine - Community
    Engine:
     Version:          19.03.6
     API version:      1.40 (minimum version 1.12)
     Go version:       go1.12.16
     Git commit:       369ce74a3c
     Built:            Thu Feb 13 01:26:21 2020
     OS/Arch:          linux/amd64
     Experimental:     false
    containerd:
     Version:          1.2.10
     GitCommit:        b34a5c8af56e510852c35414db4c1f4fa6172339
    runc:
     Version:          1.0.0-rc8+dev
     GitCommit:        3e425f80a8c931f88e6d94a8c831b9d5aa481657
    docker-init:
     Version:          0.18.0
     GitCommit:        fec3683
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401518065
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            DETECTED_DB_USER=${BASH_REMATCH[2]} &&
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            DETECTED_DB_SCHEMA=${BASH_REMATCH[5]} &&
+            DETECTED_DB_PARAMS=${BASH_REMATCH[6]}
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW__CORE__SQL_ALCHEMY_CONN} == sqlite* ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|webserver)$ ]] || \
+        [[ ${DB_RESET:="false"} == "true" ]]; then
+    if [[ ${AIRFLOW_VERSION} == *1.10* ]]; then
 
 Review comment:
   Good catch @ashb 

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.49%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   86.67%   -0.50%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39168     -223     
   - Misses       5796     6019     +223     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [4 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607852398
 
 
   Taking another look now

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402503373
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
 
 Review comment:
   Feel free to create a PR for this fixes :)

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


With regards,
Apache Git Services

[GitHub] [airflow] martin-kokos commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
martin-kokos commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400462180
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
+
+Those variables should be overwritten only if the production image is built from local sources
+rather than from GitHub repository.
+
+* COPY_SOURCE - should be set to "." if you the image is built from sources
+* COPY_TARGET - should be set to a airflow source directory inside container if the image is built
+  from sources
+* AIRFLOW_SOURCES - should point to the same as COPY_TARGET in case production image is installed from
+  the local sources rather than from Github repository. If left empty it points to sources from
+  GitHub repository derived from AIRFLOW_ORG, AIRFLOW_REPO, AIRFLOW_GIT_REFERENCE
+* CONSTRAINT_REQUIREMENTS should point to locally available requirements file if the image is built
+  from sources. If left empty it will point to the right requirement file from GitHub repository
+  derived from AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+* ENTRYPOINT_FILE - sources point to entrypoint.sh file when installing from sources if left empty
+  it points to sources from GitHub repository derived from
+  AIRFLOW_ORG, AIRFLOW_REPO, REQUIREMENTS_GIT_REFERENCE
+
+The below will production image in version 3.6 with default extras from master branch in Github:
+
+.. code-block::
+
+  docker build .
+
+The below will build the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github. Note that versions 1.10.9 and below
+have no requirements so requirements should be taken from head of the v1-10-test branch. Once we
+release 1.10.10 we can take them from the 1.10.10 tag.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg AIRFLOW_GIT_REFERENCE=1.10.9 \
+    --build-arg REQUIREMENTS_GIT_REFERENCE=v1-10-test --build-arg WWW_FOLDER="www_rbac"
+
+The below will build the production image in version 3.7 with default extras from master branch in Github.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+The below will build the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
 
 Review comment:
   A starting double-quote fell out. I appreciate your work. Thank you.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402371533
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,333 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
 
 Review comment:
   Do you see any security problems with it? Or do you have other concerns?

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399812108
 
 

 ##########
 File path: .travis.yml
 ##########
 @@ -148,6 +148,20 @@ jobs:
         INSTALL_AIRFLOW_VERSION="1.10.9"
       stage: test
       script: ./scripts/ci/ci_prepare_and_test_backport_packages.sh
+    - name: "Build production image Py3.6"
+      env: >-
+        PYTHON_MAJOR_MINOR_VERSION="3.6"
+      stage: test
+      script: ./scripts/ci/ci_build_production_image.sh
+      before_install:
+        - echo
+    - name: "Build production image Py3.7"
+      env: >-
+        PYTHON_MAJOR_MINOR_VERSION="3.7"
+      stage: test
+      script: ./scripts/ci/ci_build_production_image.sh
+      before_install:
+        - echo
 
 Review comment:
   Should we build the image every time? 

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.26%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   86.79%   +0.26%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39353     +146     
   + Misses       6105     5989     -116     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [31 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...602a289](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402400387
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,430 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
 
 Review comment:
   Should we add here the information that appears later on?
   ```
   Note! Breeze by default builds production image from local sources. You can change it's behaviour by
    providing ``--install-airflow-version``
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401824400
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
 
 Review comment:
   BTW. I think the reason I had them exported was because user/schema/params are not used and spellcheck complained. So I quickly exported them but then forgot to solve it properly. I solved it now by changing them to local and leaving comments in place of not used params.

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


With regards,
Apache Git Services

[GitHub] [airflow] xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396835351
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+COPY --chown="50000:50000" --from=airflow-build-image \
 
 Review comment:
   @potiuk 
   I can build the image with
   ```
   COPY --chown=${AIRFLOW_GID}:${AIRFLOW_UID} --from=airflow-build-image \
           /root/.local "/home/${AIRFLOW_USER}/.local"
   ```
   
   Is it something to do with the docker engine version? My version is 
   
   ```
   Client: Docker Engine - Community
    Version:           19.03.6
    API version:       1.40
    Go version:        go1.12.16
    Git commit:        369ce74a3c
    Built:             Thu Feb 13 01:27:49 2020
    OS/Arch:           linux/amd64
    Experimental:      false
   
   Server: Docker Engine - Community
    Engine:
     Version:          19.03.6
     API version:      1.40 (minimum version 1.12)
     Go version:       go1.12.16
     Git commit:       369ce74a3c
     Built:            Thu Feb 13 01:26:21 2020
     OS/Arch:          linux/amd64
     Experimental:     false
    containerd:
     Version:          1.2.10
     GitCommit:        b34a5c8af56e510852c35414db4c1f4fa6172339
    runc:
     Version:          1.0.0-rc8+dev
     GitCommit:        3e425f80a8c931f88e6d94a8c831b9d5aa481657
    docker-init:
     Version:          0.18.0
     GitCommit:        fec3683
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396442904
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
 
 Review comment:
   Do we need a ``-dev`` dependencies in production?

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402501422
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,427 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+You can also build production images from PIP packages via providing ``--install-airflow-version``
+parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earlier version -  both constraint and requirements should point to 1.10.10 version.
+
+You can also build production images from specific Git version via providing ``--install-airflow-reference``
+parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from ``https://raw.githubusercontent.com/apache/airflow/<tag>/entrypoint.sh``
+url.
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
+    --build-arg WWW_FOLDER="www_rbac"
+
+This builds the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE=entrypoint.sh \
+    --build-arg AIRFLOW_INSTALL_SOURCES="apache-airflow" \
+    --build-arg AIRFLOW_INSTALL_VERSION="==1.10.10" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt"
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
+    --build-arg WWW_FOLDER="www_rbac"
 
 Review comment:
   There is sth. wrong.
   
   ```suggestion
     docker build . 
       --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
       --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
       --build-arg AIRFLOW_INSTALL_SOURCES="apache-airflow" \
       --build-arg AIRFLOW_INSTALL_VERSION="==1.10.10" \
       --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
       --build-arg ENTRYPOINT_FILE=entrypoint.sh \
       --build-arg AIRFLOW_SOURCES_FROM="Dockerfile" \
       --build-arg AIRFLOW_SOURCES_TO="/Dockerfile" \
       --build-arg WWW_FOLDER="www_rbac"
   ```
   
   Like this @potiuk ?

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **decrease** coverage by `53.71%`.
   > The diff coverage is `37.05%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #7832       +/-   ##
   ===========================================
   - Coverage   86.52%   32.80%   -53.72%     
   ===========================================
     Files         933      933               
     Lines       45312    45342       +30     
   ===========================================
   - Hits        39207    14876    -24331     
   - Misses       6105    30466    +24361     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `87.50% <ø> (-8.34%)` | :arrow_down: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `45.97% <ø> (-45.72%)` | :arrow_down: |
   | [airflow/providers/docker/operators/docker.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZG9ja2VyL29wZXJhdG9ycy9kb2NrZXIucHk=) | `21.64% <0.00%> (-75.26%)` | :arrow_down: |
   | [...s/google/cloud/example\_dags/example\_cloud\_build.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2V4YW1wbGVfZGFncy9leGFtcGxlX2Nsb3VkX2J1aWxkLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ud/example\_dags/example\_kubernetes\_engine\_setup.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2V4YW1wbGVfZGFncy9leGFtcGxlX2t1YmVybmV0ZXNfZW5naW5lX3NldHVwLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `14.19% <ø> (-72.24%)` | :arrow_down: |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `19.04% <ø> (-73.02%)` | :arrow_down: |
   | [airflow/providers/sftp/hooks/sftp.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvc2Z0cC9ob29rcy9zZnRwLnB5) | `22.68% <0.00%> (-72.17%)` | :arrow_down: |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `16.66% <16.66%> (-45.84%)` | :arrow_down: |
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `30.95% <20.00%> (-64.29%)` | :arrow_down: |
   | ... and [818 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...3b336c6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.64%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   87.17%   +0.64%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39526     +319     
   + Misses       6105     5816     -289     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [35 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...3b336c6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402407041
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   Without .pyc:
   
   Airflow Layer = 419 MB, Compressed size = 99.44 MB
   Total Image size = 837 MB. Compressed size = 237.2 MB
   
   With .pyc:
   
   Airflow Layer = 516 MB, Compressed size = 125 MB
   Total image = 933 MB, Compressed size = 263.17 MB
   
   So we save 26 MB of transfer and almost 100 MB of the image layer after decompression - 10% more or less of both compressed and uncompressed size. 
   
   I think this is quite significant saving. 
   
   I also measured startup time with and without .pyc and for webserver it was around a second (hard to say exactly).
   
   and yes the PYTHONDONTWRITEBYTECODE did not work for install from sources. not sure yet why.
   
   
   

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


With regards,
Apache Git Services

[GitHub] [airflow] dimberman commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
dimberman commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607974554
 
 
   @potiuk I can't tell if I need to reload something but I'm not seeing build-image in auto-complete ```
   (airflow) dimberman airflow ((HEAD detached at polidea/add-production-docker-image)) (⎈ |N/A|N/A)
    $ ./breeze build-
   build-docs  build-only
   (airflow) dimberman airflow ((HEAD detached at polidea/add-production-docker-image)) (⎈ |N/A|N/A)
    $ ./breeze build-image --production-image
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402392466
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
 
 Review comment:
   It's all on us. We just make a PR with hash of the commit from repo and they approve it.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402411621
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,430 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
 
 Review comment:
   This is the same parameter as used everywhere. It's limitations are shown in BREEZE.rst, `./breeze flags`  and autocomplete. tt's also verified when the value is not ok (and correct values are shown). I do not want to repeat it here again, especially that now we have one place that it is changed and Breeze documentation is generated from it. 

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401512033
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            DETECTED_DB_USER=${BASH_REMATCH[2]} &&
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            DETECTED_DB_SCHEMA=${BASH_REMATCH[5]} &&
+            DETECTED_DB_PARAMS=${BASH_REMATCH[6]}
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW__CORE__SQL_ALCHEMY_CONN} == sqlite* ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|webserver)$ ]] || \
+        [[ ${DB_RESET:="false"} == "true" ]]; then
+    if [[ ${AIRFLOW_VERSION} == *1.10* ]]; then
 
 Review comment:
   Aaaaargh!

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


With regards,
Apache Git Services

[GitHub] [airflow] xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396835351
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+COPY --chown="50000:50000" --from=airflow-build-image \
 
 Review comment:
   @potiuk 
   I can build the image with
   ```
   COPY --chown=${AIRFLOW_GID}:${AIRFLOW_UID} --from=airflow-build-image \
           /root/.local "/home/${AIRFLOW_USER}/.local"
   ```
   
   Is it something to do with the docker engine version? My version is 
   
   ```
   Client: Docker Engine - Community
    Version:           19.03.6
    API version:       1.40
    Go version:        go1.12.16
    Git commit:        369ce74a3c
    Built:             Thu Feb 13 01:27:49 2020
    OS/Arch:           linux/amd64
    Experimental:      false
   
   Server: Docker Engine - Community
    Engine:
     Version:          19.03.6
     API version:      1.40 (minimum version 1.12)
     Go version:       go1.12.16
     Git commit:       369ce74a3c
     Built:            Thu Feb 13 01:26:21 2020
     OS/Arch:          linux/amd64
     Experimental:     false
    containerd:
     Version:          1.2.10
     GitCommit:        b34a5c8af56e510852c35414db4c1f4fa6172339
    runc:
     Version:          1.0.0-rc8+dev
     GitCommit:        3e425f80a8c931f88e6d94a8c831b9d5aa481657
    docker-init:
     Version:          0.18.0
     GitCommit:        fec3683
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400889199
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
 
 Review comment:
   WIP? 3x the same code block @potiuk 

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r397002662
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+# Note that we have to hard-code user id/group id as you cannot use
+# args nor variables in --chown :(. That's a limitation of
+# docker build.
+COPY --chown="50000:50000" --from=airflow-build-image \
+        /root/.local "/home/${AIRFLOW_USER}/.local"
 
 Review comment:
   Will try . Thanks !

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-606856606
 
 
   Looking

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/51be6ec1c0f430d964c3f32895a9f909d3ff8300&el=desc) will **decrease** coverage by `22.40%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #7832       +/-   ##
   ===========================================
   - Coverage   87.07%   64.67%   -22.41%     
   ===========================================
     Files         932      932               
     Lines       45187    45191        +4     
   ===========================================
   - Hits        39348    29228    -10120     
   - Misses       5839    15963    +10124     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/S3\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9TM19ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/pig\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9waWdfaG9vay5weQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/hdfs\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9oZGZzX2hvb2sucHk=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/http\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9odHRwX2hvb2sucHk=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/jdbc\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9qZGJjX2hvb2sucHk=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/druid\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kcnVpZF9ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/hive\_hooks.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9oaXZlX2hvb2tzLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/mssql\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9tc3NxbF9ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/mysql\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9teXNxbF9ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/samba\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9zYW1iYV9ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [503 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [51be6ec...81e6c3a](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/6bf3058c94527dd59574cf2ce415756feab1ef99&el=desc) will **decrease** coverage by `0.51%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.30%   86.78%   -0.52%     
   ==========================================
     Files         935      935              
     Lines       45384    45392       +8     
   ==========================================
   - Hits        39621    39395     -226     
   - Misses       5763     5997     +234     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [6bf3058...f0ee1c7](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] zhongjiajie commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
zhongjiajie commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402456764
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,332 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG WWW_FOLDER="www"
+
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+ARG AIRFLOW_SOURCES_TO="/opt/airflow"
+ENV AIRFLOW_SOURCES_TO=${AIRFLOW_SOURCES_TO}
+
+COPY ${AIRFLOW_SOURCES_FROM} ${AIRFLOW_SOURCES_TO}
+
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ARG AIRFLOW_EXTRAS
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ARG AIRFLOW_INSTALL_SOURCES="."
+ENV AIRFLOW_INSTALL_SOURCES=${AIRFLOW_INSTALL_SOURCES}
+
+ARG AIRFLOW_INSTALL_VERSION=""
+ENV AIRFLOW_INSTALL_VERSION=${AIRFLOW_INSTALL_VERSION}
+
+ARG CONSTRAINT_REQUIREMENTS="requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG AIRFLOW_SOURCES_FROM="."
+ENV AIRFLOW_SOURCES_FROM=${AIRFLOW_SOURCES_FROM}
+
+WORKDIR /opt/airflow
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_INSTALL_SOURCES}[${AIRFLOW_EXTRAS}]${AIRFLOW_INSTALL_VERSION}" \
+    --constraint /requirements.txt && \
+    find /root/.local/ -name '*.pyc' -print0 | xargs -0 rm -r && \
+    find /root/.local/ -type d -name '__pycache__' -print0 | xargs -0 rm -r
+
+
+ARG WWW_FOLDER
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN if [[ -f "${AIRFLOW_WWW}/package.json" ]]; then \
+        yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache; \
+        yarn --cwd ${AIRFLOW_WWW} run prod; \
+        rm -rf ${AIRFLOW_WWW}/node_modules; \
+    fi
+
+ARG ENTRYPOINT_FILE="entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG PYTHON_BASE_IMAGE
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+
+ARG AIRFLOW_VERSION
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi6 \
+           libsasl2-2 \
+           libsasl2-modules \
+           libssl1.1 \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           rsync \
+           sasl2-bin \
+           sqlite3 \
+           sudo \
+           unixodbc \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient20 \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+ARG PIP_VERSION
+ENV PIP_VERSION=${PIP_VERSION}
+RUN pip install --upgrade pip==${PIP_VERSION}
 
 Review comment:
   Could we add one more stage, for the same install, just like install/upgrade pip, and install mysql-client?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401511427
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,389 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default, the images are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Environment variable                     | Default value                            | Description                              |
 
 Review comment:
   Yep!

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605626827
 
 
   > > /Users/kamilbregula/devel/google-airflow/airflow/scripts/ci/_utils.sh: line 628: AIRFLOW_REMOTE_MANIFEST_IMAGE: unbound variable
   Fixed.
   

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


With regards,
Apache Git Services

[GitHub] [airflow] feluelle edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
feluelle edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607977704
 
 
   @dimberman run `breeze-complete` script again. (`. ./breeze-complete` did it for me)

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/6bf3058c94527dd59574cf2ce415756feab1ef99&el=desc) will **decrease** coverage by `0.67%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.30%   86.62%   -0.68%     
   ==========================================
     Files         935      935              
     Lines       45384    45392       +8     
   ==========================================
   - Hits        39621    39323     -298     
   - Misses       5763     6069     +306     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/api/auth/backend/kerberos\_auth.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9hcGkvYXV0aC9iYWNrZW5kL2tlcmJlcm9zX2F1dGgucHk=) | `28.16% <0.00%> (-54.93%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/security/kerberos.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9zZWN1cml0eS9rZXJiZXJvcy5weQ==) | `30.43% <0.00%> (-45.66%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | ... and [10 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [6bf3058...f0ee1c7](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396834360
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
 
 Review comment:
   This will get swamped by the installation log really quickly, so is this necessary for development purpose? If it is for providing a summary after the build, we can push this down near the `CMD` command

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396439574
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
 
 Review comment:
   Aahh. Right.

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400868281
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
 
 Review comment:
   ```suggestion
     .. 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.
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396433481
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   Should we remove this variable after installation?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399171768
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -348,18 +349,43 @@ In case of disk space errors on macOS, increase the disk space available for Doc
 Building the Images
 -------------------
 
-You can manually trigger building the local images using the script:
+You can manually trigger building the local CI image using this command:
 
 .. code-block::
 
-  ./breeze build-only
+  ./breeze build-ci-image
 
-The scripts that build the images are optimized to minimize the time needed to rebuild the image when
+And production image using this command:
+
+.. code-block::
+
+  ./breeze build-prod-image
+
+
+The images are build with default extras - different for CI image and different for production image
+and using python version chosen in breeze. You can change those parameters by specifying them as
+command line options. You can see default extras used via  ``./breeze flags`` or in the syntax example
+below.
+
+For example if you want to build python 3.7 version of production image with
+all extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-prod-image --python 3.7 --extras "all"
 
 Review comment:
   IT will work  in one combination and it is slower because it is multi-segmented image which cannot be cached directly. If you want to reflect the actual build command it should be something like 
   
   ```
   docker build . --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg
   AIRFLOW_EXTRAS="all" --cache-from apache/airflow:master-build-3.7 --cache-from 
   apache/airflow:master-3.7 
   ```
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/51be6ec1c0f430d964c3f32895a9f909d3ff8300&el=desc) will **decrease** coverage by `22.38%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #7832       +/-   ##
   ===========================================
   - Coverage   87.07%   64.69%   -22.39%     
   ===========================================
     Files         932      932               
     Lines       45187    45191        +4     
   ===========================================
   - Hits        39348    29236    -10112     
   - Misses       5839    15955    +10116     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/S3\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9TM19ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/pig\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9waWdfaG9vay5weQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/hdfs\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9oZGZzX2hvb2sucHk=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/http\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9odHRwX2hvb2sucHk=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/jdbc\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9qZGJjX2hvb2sucHk=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/druid\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9kcnVpZF9ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/hive\_hooks.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9oaXZlX2hvb2tzLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/mssql\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9tc3NxbF9ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/mysql\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9teXNxbF9ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [airflow/hooks/samba\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9zYW1iYV9ob29rLnB5) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [502 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [51be6ec...81e6c3a](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607582286
 
 
   All corrected - please take another look

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401823282
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
 
 Review comment:
   Yep. Reverted the logic and it looks much more reasonable.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400502678
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
 
 Review comment:
   Oh nm, I see this is the build image.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401203890
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,389 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default, the images are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Environment variable                     | Default value                            | Description                              |
 
 Review comment:
   ```suggestion
   | Build arg                     | Default value                            | Description                              |
   ```
   
   Since these are controlled by `docker build --build-arg` aren't they?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396944135
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
 
 Review comment:
   It's only during the build not the installation. I just wanted to make sure that during the build we see which parameters are used to build it. So during installation none of this is printed :)

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399789265
 
 

 ##########
 File path: BREEZE.rst
 ##########
 @@ -348,18 +349,43 @@ In case of disk space errors on macOS, increase the disk space available for Doc
 Building the Images
 -------------------
 
-You can manually trigger building the local images using the script:
+You can manually trigger building the local CI image using this command:
 
 .. code-block::
 
-  ./breeze build-only
+  ./breeze build-ci-image
 
-The scripts that build the images are optimized to minimize the time needed to rebuild the image when
+And production image using this command:
+
+.. code-block::
+
+  ./breeze build-prod-image
+
+
+The images are build with default extras - different for CI image and different for production image
+and using python version chosen in breeze. You can change those parameters by specifying them as
+command line options. You can see default extras used via  ``./breeze flags`` or in the syntax example
+below.
+
+For example if you want to build python 3.7 version of production image with
+all extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-prod-image --python 3.7 --extras "all"
 
 Review comment:
   I added a lot of docs already. But wait for next push - I separated information about both CI and Production images to separate IMAGES.rst file. Bear with me!

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401381168
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
 
 Review comment:
   > Official docker images are so you can do `docker pull airflow`? I'm not sure AFS would like us to drop the Apache name there.
   
   I am definitely planning to pass it through PMC (and ASF if we so decide) before I make a submission. 
   
   And I would actually prefer "apache-airflow" as a name (same as we use for pip package) so it will rather be `docker pull apache-airflow`. I think it's good if you read the guide here about official images here: https://docs.docker.com/docker-hub/official_images/. It was originally intended for languages but then other projects started using it which were more like frameworks (django for one).  It is followed by python, nodejs or django do (see https://hub.docker.com/_/django/) - those are all free foundations-managed projects and the idea with this program is that it certifies that image is maintained by some organization (which it will be in our case). It is prominently displayed, it goes through automated vulnerability scanning for example. I think it's good to have our image as official one.
   
   Just a note - the intention to do that  was clear from the very beginning see already approved AIP-26: https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-26+Production-ready+Airflow+Docker+Image+and+helm+chart - it's point no. 5) in the acceptance criteria.
   
   Voting here: https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-26+Production-ready+Airflow+Docker+Image+and+helm+chart
   
   
   
   
   
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401206953
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
 
 Review comment:
   ```suggestion
       DETECTED_DB_BACKEND=""
       DETECTED_DB_USER=""
       DETECTED_DB_HOST=""
       DETECTED_DB_PORT=""
       DETECTED_DB_SCHEMA=""
       DETECTED_DB_PARAMS=""
   ```
   
   These aren't used by any subcommands, so no need to export them.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401204155
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,389 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default, the images are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Environment variable                     | Default value                            | Description                              |
 
 Review comment:
   Or `Argument` as you have it in the next block.

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400503809
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,375 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
+ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+ARG CONSTRAINT_REQUIREMENTS="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/requirements/requirements-python${PYTHON_MAJOR_MINOR_VERSION}.txt"
+ENV CONSTRAINT_REQUIREMENTS=${CONSTRAINT_REQUIREMENTS}
+
+ARG ENTRYPOINT_FILE="${AIRFLOW_RAW_CONTENT_URL}/${REQUIREMENTS_GIT_REFERENCE}/entrypoint.sh"
+ENV ENTRYPOINT_FILE="${ENTRYPOINT_FILE}"
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "Airflow install source: ${AIRFLOW_SOURCES}"; \
+    echo "Constraint requirements: ${CONSTRAINT_REQUIREMENTS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Install curl and gnupg2 - needed to download nodejs in the next step
+RUN apt-get update \
+    && apt-get install -y --no-install-recommends \
+           curl \
+           gnupg2 \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install basic apt dependencies
+RUN curl --fail --location https://deb.nodesource.com/setup_10.x | bash - \
+    && curl https://dl.yarnpkg.com/debian/pubkey.gpg | apt-key add - > /dev/null \
+    && echo "deb https://dl.yarnpkg.com/debian/ stable main" > /etc/apt/sources.list.d/yarn.list \
+    # Note missing man directories on debian-buster
+    # https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+    && mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           nodejs \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+           yarn \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation (only in build image - it does not affect the main image)
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+# We do not need it here technially but we are going to
+# override COPY_SOURCE with "." and COPY_TARGET with "/opt/airflow"
+# this will let us build the prod image from sources rather than
+# from remote URL from GitHub
+ARG COPY_SOURCE=Dockerfile
+ARG COPY_TARGET=/Dockerfile
+
+## Only needed for installing from sources
+COPY ${COPY_SOURCE} ${COPY_TARGET}
+
+# hadolint ignore=DL3020
+ADD "${CONSTRAINT_REQUIREMENTS}" /requirements.txt
+
+ENV PATH=${PATH}:/root/.local/bin
+
+RUN pip install --user "${AIRFLOW_SOURCES}[${AIRFLOW_EXTRAS}]" \
+    --constraint /requirements.txt
+
+ENV WWW_FOLDER=${WWW_FOLDER}
+
+ENV AIRFLOW_WWW=/root/.local/lib/python${PYTHON_MAJOR_MINOR_VERSION}/site-packages/airflow/${WWW_FOLDER}
+
+RUN yarn --cwd ${AIRFLOW_WWW} install --frozen-lockfile --no-cache \
+    && yarn --cwd ${AIRFLOW_WWW} run prod \
+    && rm -rf ${AIRFLOW_WWW}/node_modules
+
+# hadolint ignore=DL3020
+ADD ${ENTRYPOINT_FILE} /entrypoint
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+LABEL org.apache.airflow.gid="${AIRFLOW_GID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
 
 Review comment:
   Not needed in "main" image is it?

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r399789030
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
 
 Review comment:
   Apparently that was needed to add python2-stdlib (someone added it for that purpose in CI image). When I removed it - some tests started to fail (virtualenv) and the reason was that python2-stdlib was not installed. I installed it instead of python-selinux and all tests started to work.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396438551
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
 
 Review comment:
   I added some explanation in comments about it.

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396937848
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,325 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
 
 Review comment:
   Do we need to install MySQL second time? We did the same in base image

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


With regards,
Apache Git Services

[GitHub] [airflow] mik-laj commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396454988
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,322 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALFA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        nuch smaller.
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN pip install --user \
+    "${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow[${AIRFLOW_EXTRAS}]" \
+    --constraint  "${AIRFLOW_RAW_CONTENT_URL}/${AIRFLOW_GIT_REFERENCE}/requirements.txt"
+
+##############################################################################################
+# This is the actual Airflow image - much smaller than the build one. We copy
+# installed Airflow and all it's dependencies from the build image to make it smaller.
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as main
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_USER
+ARG AIRFLOW_GROUP
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+
+ENV AIRFLOW_HOME=${AIRFLOW_HOME}
+ENV AIRFLOW_USER=${AIRFLOW_USER}
+ENV AIRFLOW_GROUP=${AIRFLOW_GROUP}
+ENV AIRFLOW_UID=${AIRFLOW_UID}
+ENV AIRFLOW_GID=${AIRFLOW_GID}
+
+ENV PIP_VERSION=${PIP_VERSION}
+
+# Print versions
+RUN echo "Building main airflow image"; \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow home: ${AIRFLOW_HOME}"; \
+    echo "Airflow user: ${AIRFLOW_USER}"; \
+    echo "Airflow uid: ${AIRFLOW_UID}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           ca-certificates \
+           curl \
+           dumb-init \
+           freetds-bin \
+           freetds-dev \
+           gnupg \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           netcat \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+RUN pip install --upgrade pip==${PIP_VERSION}
+
+RUN addgroup --gid "${AIRFLOW_GID}" "${AIRFLOW_GROUP}" && \
+    adduser --quiet "${AIRFLOW_USER}" --uid "${AIRFLOW_UID}" \
+        --ingroup "${AIRFLOW_GROUP}" \
+        --home /home/${AIRFLOW_USER}
+
+RUN mkdir -pv ${AIRFLOW_HOME}; \
+    mkdir -pv ${AIRFLOW_HOME}/dags; \
+    mkdir -pv ${AIRFLOW_HOME}/logs; \
+    chown -R "${AIRFLOW_USER}" ${AIRFLOW_HOME}
+
+COPY --chown="50000:50000" --from=airflow-build-image \
 
 Review comment:
   ```suggestion
   COPY --chown="${AIRFLOW_GID}:${AIRFLOW_UID}" --from=airflow-build-image \
   ```

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/0f19a930d1a7dec2a96bab0de144829f83cc0626&el=desc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.17%   87.16%   -0.01%     
   ==========================================
     Files         932      932              
     Lines       45187    45187              
   ==========================================
   - Hits        39391    39389       -2     
   - Misses       5796     5798       +2     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/utils/dag\_processing.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9kYWdfcHJvY2Vzc2luZy5weQ==) | `87.94% <0.00%> (-0.38%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [0f19a93...15283a6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-607739577
 
 
   Hey @ashb @kaxil - I want to merge it to 1.10.10 before we release, Is there anything else you want to comment on?

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/51be6ec1c0f430d964c3f32895a9f909d3ff8300&el=desc) will **decrease** coverage by `53.64%`.
   > The diff coverage is `41.34%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff             @@
   ##           master    #7832       +/-   ##
   ===========================================
   - Coverage   87.16%   33.52%   -53.65%     
   ===========================================
     Files         932      933        +1     
     Lines       45187    45342      +155     
   ===========================================
   - Hits        39389    15201    -24188     
   - Misses       5798    30141    +24343     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/cli/commands/db\_command.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY29tbWFuZHMvZGJfY29tbWFuZC5weQ==) | `30.95% <0.00%> (-64.29%)` | :arrow_down: |
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `87.50% <ø> (+0.83%)` | :arrow_up: |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `45.97% <ø> (-45.51%)` | :arrow_down: |
   | [airflow/providers/docker/operators/docker.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZG9ja2VyL29wZXJhdG9ycy9kb2NrZXIucHk=) | `21.64% <0.00%> (-75.23%)` | :arrow_down: |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `14.19% <ø> (-72.24%)` | :arrow_down: |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `19.04% <ø> (-73.02%)` | :arrow_down: |
   | [airflow/providers/sftp/hooks/sftp.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvc2Z0cC9ob29rcy9zZnRwLnB5) | `22.68% <0.00%> (-72.17%)` | :arrow_down: |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `16.66% <16.66%> (ø)` | |
   | [...iders/google/marketing\_platform/hooks/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9ob29rcy9hbmFseXRpY3MucHk=) | `25.45% <20.00%> (-74.55%)` | :arrow_down: |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `26.99% <26.66%> (-65.49%)` | :arrow_down: |
   | ... and [809 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...d93bf3c](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.64%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   87.17%   +0.64%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39526     +319     
   + Misses       6105     5816     -289     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [35 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...3b336c6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] ashb commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401198373
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,373 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS ALPHA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+#
+# This is a multi-segmented image. It actually contains two images:
+#
+# airflow-build-image  - there all airflow dependencies can be installed (and
+#                        built - for those dependencies that require
+#                        build essentials). Airflow is installed there with
+#                        --user switch so that all the dependencies are
+#                        installed to ${HOME}/.local
+#
+# main                 - this is the actual production image that is much
+#                        smaller because it does not contain all the build
+#                        essentials. Instead the ${HOME}/.local folder
+#                        is copied from the build-image - this way we have
+#                        only result of installation and we do not need
+#                        all the build essentials. This makes the image
+#                        much smaller.
+#
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG REQUIREMENTS_GIT_REFERENCE="master"
+ARG WWW_FOLDER="www"
+ARG AIRFLOW_EXTRAS="async,aws,azure,celery,dask,elasticsearch,gcp,kubernetes,mysql,postgres,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+ARG PYTHON_MAJOR_MINOR_VERSION="3.6"
+
+##############################################################################################
+# This is the build image where we build all dependencies
+##############################################################################################
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG REQUIREMENTS_GIT_REFERENCE
+ARG WWW_FOLDER
+ARG AIRFLOW_EXTRAS
+
+ARG AIRFLOW_HOME
+ARG AIRFLOW_UID
+ARG AIRFLOW_GID
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ARG PYTHON_MAJOR_MINOR_VERSION
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION}
+
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+ENV REQUIREMENTS_GIT_REFERENCE=${REQUIREMENTS_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+ARG AIRFLOW_SOURCES="${AIRFLOW_REPO_URL}/archive/${AIRFLOW_GIT_REFERENCE}.tar.gz#egg=apache-airflow"
 
 Review comment:
   Just noticed this, should have twigged earlier.
   
   I'm not sure how I feel about installing from git, rather than from the release artifacts we vote upon.

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/6bf3058c94527dd59574cf2ce415756feab1ef99&el=desc) will **decrease** coverage by `0.51%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.30%   86.78%   -0.52%     
   ==========================================
     Files         935      935              
     Lines       45384    45392       +8     
   ==========================================
   - Hits        39621    39395     -226     
   - Misses       5763     5997     +234     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [6bf3058...f0ee1c7](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/6bf3058c94527dd59574cf2ce415756feab1ef99&el=desc) will **decrease** coverage by `0.51%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   - Coverage   87.30%   86.78%   -0.52%     
   ==========================================
     Files         935      935              
     Lines       45384    45392       +8     
   ==========================================
   - Hits        39621    39395     -226     
   - Misses       5763     5997     +234     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...flow/providers/apache/cassandra/hooks/cassandra.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2Nhc3NhbmRyYS9ob29rcy9jYXNzYW5kcmEucHk=) | `21.25% <0.00%> (-72.50%)` | :arrow_down: |
   | [...w/providers/apache/hive/operators/mysql\_to\_hive.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvYXBhY2hlL2hpdmUvb3BlcmF0b3JzL215c3FsX3RvX2hpdmUucHk=) | `35.84% <0.00%> (-64.16%)` | :arrow_down: |
   | [airflow/providers/postgres/operators/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvb3BlcmF0b3JzL3Bvc3RncmVzLnB5) | `47.82% <0.00%> (-52.18%)` | :arrow_down: |
   | [airflow/providers/redis/operators/redis\_publish.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvb3BlcmF0b3JzL3JlZGlzX3B1Ymxpc2gucHk=) | `50.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [airflow/providers/mongo/sensors/mongo.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbW9uZ28vc2Vuc29ycy9tb25nby5weQ==) | `53.33% <0.00%> (-46.67%)` | :arrow_down: |
   | [airflow/providers/mysql/operators/mysql.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvbXlzcWwvb3BlcmF0b3JzL215c3FsLnB5) | `55.00% <0.00%> (-45.00%)` | :arrow_down: |
   | [airflow/providers/redis/sensors/redis\_key.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcmVkaXMvc2Vuc29ycy9yZWRpc19rZXkucHk=) | `61.53% <0.00%> (-38.47%)` | :arrow_down: |
   | [airflow/executors/celery\_executor.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9leGVjdXRvcnMvY2VsZXJ5X2V4ZWN1dG9yLnB5) | `50.67% <0.00%> (-37.84%)` | :arrow_down: |
   | [...roviders/google/cloud/operators/postgres\_to\_gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL29wZXJhdG9ycy9wb3N0Z3Jlc190b19nY3MucHk=) | `54.28% <0.00%> (-31.43%)` | :arrow_down: |
   | [airflow/providers/postgres/hooks/postgres.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvcG9zdGdyZXMvaG9va3MvcG9zdGdyZXMucHk=) | `79.16% <0.00%> (-15.28%)` | :arrow_down: |
   | ... and [5 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [6bf3058...f0ee1c7](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401021259
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
+* ARG AIRFLOW_REPO=apache/airflow - the repository from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_BRANCH=master - the branch from which PIP dependencies are installed (CI optimised)
+* ARG AIRFLOW_CI_BUILD_EPOCH="1" - increasing this value will reinstall PIP dependencies from the repository
+      from scratch
+* ARG AIRFLOW_EXTRAS="all" - extras to install
+* ARG ADDITIONAL_PYTHON_DEPS="" - additional python dependencies to install
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+This builds the CI image in version 3.7 with just  extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+Production images
+.................
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG AIRFLOW_ORG="apache" - Github organisation from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_REPO="airflow" - Github repository from which Airflow is installed (when installed from repo)
+* ARG AIRFLOW_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    Airflow is installed (when installed from repo)
+* ARG REQUIREMENTS_GIT_REFERENCE="master" - reference (branch or tag) from Github repository from which
+    requirements are downloaded for constraints (when installed from repo).
+* ARG WWW_FOLDER="www" - folder where www pages are generated - it should be set to www_rbac in case
+    of 1.10 image builds.
+* ARG AIRFLOW_EXTRAS=(see Dockerfile)  Default extras with which airflow is installed
+* ARG AIRFLOW_HOME=/opt/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_UID="50000" - Airflow user UID
+* ARG AIRFLOW_GID="50000" - Airflow group GID
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG CASS_DRIVER_BUILD_CONCURRENCY="8" - Number of processors to use for cassandra PIP install (speeds up
+       installing in case cassandra extra is used).
 
 Review comment:
   Thanks!

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


With regards,
Apache Git Services

[GitHub] [airflow] kaxil commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r400871747
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,304 @@
+
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airlfow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image are optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that build the production image are optimised for size of the image.
+
+In Breeze by default the images with are built using local sources of Apache Airflow. However
+you can also build production images from github sources - providing ``--install-airflow-version``
+parameter to Breeze. This will install airflow inside the production image using sources downloaded from
+specified tag or branch. Internally airflow will be installed using the command:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following arguments can be used for CI images:
+
+* ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster" - Base python image
+* ARG AIRFLOW_VERSION="2.0.0.dev0" - version of Airflow
+* ARG PYTHON_MAJOR_MINOR_VERSION="3.6" - major/minor version of Python (should match base image)
+* ARG DEPENDENCIES_EPOCH_NUMBER="2" - increasing this number will reinstall all apt dependencies
+* ARG KUBECTL_VERSION="v1.15.3" - version of kubectl installed
+* ARG KIND_VERSION="v0.6.1" - version of kind installed
+* ARG PIP_NO_CACHE_DIR="true" - if true, then no pip cache will be stored
+* ARG PIP_VERSION="19.0.2" - version of PIP to use
+* ARG HOME=/root - Home directory of the root user (CI image has root user as default)
+* ARG AIRFLOW_HOME=/root/airflow - Airflow's HOME (that's where logs and sqlite databases are stored)
+* ARG AIRFLOW_SOURCES=/opt/airflow - Mounted sources of Airflow
+* ARG PIP_DEPENDENCIES_EPOCH_NUMBER="3" - increasing that number will reinstall all PIP dependencies
+* ARG CASS_DRIVER_NO_CYTHON="1" - if set to 1 no CYTHON compilation is done for cassandra driver (much faster)
+* ARG AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="true" if set then PIP dependencies are installed from repo first
+      before they are reinstalled from local sources. This allows for incremental faster builds when
+      requirements change
 
 Review comment:
   ![image](https://user-images.githubusercontent.com/8811558/78026294-48423600-7353-11ea-9d09-788c86e8a000.png)
   
   This looks weird. Maybe use backticks to wrap the args

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: [WIP] Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: [WIP] Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r396439871
 
 

 ##########
 File path: Dockerfile
 ##########
 @@ -0,0 +1,298 @@
+# 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.
+#
+# THIS DOCKERFILE IS INTENDED FOR PRODUCTION USE AND DEPLOYMENT.
+# NOTE! IT IS BETA-QUALITY FOR NOW - WE ARE IN A PROCESS OF TESTING IT
+#
+ARG PYTHON_BASE_IMAGE="python:3.6-slim-buster"
+
+ARG AIRFLOW_VERSION="2.0.0.dev0"
+ARG AIRFLOW_ORG="apache"
+ARG AIRFLOW_REPO="airflow"
+ARG AIRFLOW_GIT_REFERENCE="master"
+ARG AIRFLOW_EXTRAS="async,azure_blob_storage,azure_cosmos,azure_container_instances,celery,crypto,elasticsearch,gcp,kubernetes,mysql,postgres,s3,emr,redis,slack,ssh,statsd,virtualenv"
+
+ARG AIRFLOW_HOME=/opt/airflow
+ARG AIRFLOW_USER="airflow"
+ARG AIRFLOW_GROUP="airflow"
+ARG AIRFLOW_UID="50000"
+ARG AIRFLOW_GID="50000"
+
+ARG PIP_VERSION="19.0.2"
+ARG CASS_DRIVER_BUILD_CONCURRENCY="8"
+
+FROM ${PYTHON_BASE_IMAGE} as airflow-build-image
+SHELL ["/bin/bash", "-o", "pipefail", "-e", "-u", "-x", "-c"]
+
+LABEL org.apache.airflow.docker=true
+LABEL org.apache.airflow.distro="debian"
+LABEL org.apache.airflow.distro.version="buster"
+LABEL org.apache.airflow.module="airflow"
+LABEL org.apache.airflow.component="airflow"
+LABEL org.apache.airflow.image="airflow-build-image"
+LABEL org.apache.airflow.uid="${AIRFLOW_UID}"
+
+ARG AIRFLOW_VERSION
+ARG AIRFLOW_ORG
+ARG AIRFLOW_REPO
+ARG AIRFLOW_GIT_REFERENCE
+ARG AIRFLOW_EXTRAS
+
+ARG PIP_VERSION
+ARG CASS_DRIVER_BUILD_CONCURRENCY
+
+ENV PYTHON_BASE_IMAGE=${PYTHON_BASE_IMAGE}
+ENV AIRFLOW_VERSION=${AIRFLOW_VERSION}
+ENV AIRFLOW_ORG=${AIRFLOW_ORG}
+ENV AIRFLOW_REPO=${AIRFLOW_REPO}
+ENV AIRFLOW_GIT_REFERENCE=${AIRFLOW_GIT_REFERENCE}
+
+ENV AIRFLOW_EXTRAS=${AIRFLOW_EXTRAS}
+
+ENV AIRFLOW_REPO_URL="https://github.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+ENV AIRFLOW_RAW_CONTENT_URL="https://raw.githubusercontent.com/${AIRFLOW_ORG}/${AIRFLOW_REPO}"
+
+ENV PIP_VERSION=${PIP_VERSION}
+ENV CASS_DRIVER_BUILD_CONCURRENCY=${CASS_DRIVER_BUILD_CONCURRENCY}
+
+# Print versions
+RUN echo "Building airflow-build-image stage" \
+    echo "Base image: ${PYTHON_BASE_IMAGE}"; \
+    echo "Airflow version: ${AIRFLOW_VERSION}"; \
+    echo "Airflow git reference: ${AIRFLOW_GIT_REFERENCE}"; \
+    echo "Airflow org: ${AIRFLOW_ORG}"; \
+    echo "Airflow repo: ${AIRFLOW_REPO}"; \
+    echo "Airflow repo url: ${AIRFLOW_REPO_URL}"; \
+    echo "Airflow extras: ${AIRFLOW_EXTRAS}" ;\
+    echo "PIP version: ${PIP_VERSION}" ;\
+    echo "Cassandra concurrency: ${CASS_DRIVER_BUILD_CONCURRENCY}" ;\
+    echo
+
+# Make sure noninteractive debian install is used and language variables set
+ENV DEBIAN_FRONTEND=noninteractive LANGUAGE=C.UTF-8 LANG=C.UTF-8 LC_ALL=C.UTF-8 \
+    LC_CTYPE=C.UTF-8 LC_MESSAGES=C.UTF-8
+
+# Note missing man directories on debian-buster
+# https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=863199
+# Install basic apt dependencies
+RUN mkdir -pv /usr/share/man/man1 \
+    && mkdir -pv /usr/share/man/man7 \
+    && apt-get update \
+    && apt-get install -y --no-install-recommends \
+           apt-transport-https \
+           apt-utils \
+           build-essential \
+           ca-certificates \
+           curl \
+           gnupg \
+           dirmngr \
+           freetds-bin \
+           freetds-dev \
+           gosu \
+           krb5-user \
+           ldap-utils \
+           libffi-dev \
+           libkrb5-dev \
+           libpq-dev \
+           libsasl2-2 \
+           libsasl2-dev \
+           libsasl2-modules \
+           libssl-dev \
+           locales  \
+           lsb-release \
+           openssh-client \
+           postgresql-client \
+           python-selinux \
+           sasl2-bin \
+           software-properties-common \
+           sqlite3 \
+           sudo \
+           unixodbc \
+           unixodbc-dev \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean \
+    && rm -rf /var/lib/apt/lists/*
+
+# Install MySQL client from Oracle repositories (Debian installs mariadb)
+RUN KEY="A4A9406876FCBD3C456770C88C718D3B5072E1F5" \
+    && GNUPGHOME="$(mktemp -d)" \
+    && export GNUPGHOME \
+    && for KEYSERVER in $(shuf -e \
+            ha.pool.sks-keyservers.net \
+            hkp://p80.pool.sks-keyservers.net:80 \
+            keyserver.ubuntu.com \
+            hkp://keyserver.ubuntu.com:80 \
+            pgp.mit.edu) ; do \
+          gpg --keyserver "${KEYSERVER}" --recv-keys "${KEY}" && break || true ; \
+       done \
+    && gpg --export "${KEY}" | apt-key add - \
+    && gpgconf --kill all \
+    rm -rf "${GNUPGHOME}"; \
+    apt-key list > /dev/null \
+    && echo "deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7" | tee -a /etc/apt/sources.list.d/mysql.list \
+    && apt-get update \
+    && apt-get install --no-install-recommends -y \
+        libmysqlclient-dev \
+        mysql-client \
+    && apt-get autoremove -yqq --purge \
+    && apt-get clean && rm -rf /var/lib/apt/lists/*
+
+# disable bytecode generation
+ENV PYTHONDONTWRITEBYTECODE=1
 
 Review comment:
   Again - this is build image not the main image (see below comments separating those two images). This variable is only set for the build image, not for the main one. Thanks to that the .local folder is much smaller to copy - and the cache will be generated on the first run of python interpreter for the main image.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402414814
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,430 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
 
 Review comment:
   Good point!

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605688716
 
 
   This is heavily improved version of the prod image:
   - I can run it from either local sources or using releases from github
   - added a lot of documentation in IMAGES.rst
   - added yarn steps to have optimized javascript code (still need to test it)
   - Breeze is updated so that you should be able now build, run, exec either CI or prod image using  the same command ('breeze, breeze exec, breeze build-image [--production-image]).
   
   I think it's almost ready to be merged as "alpha quality" image. I will make some more tests,  but I would love more review comments.

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401556392
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
 
 Review comment:
   :D No worries. 

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


With regards,
Apache Git Services

[GitHub] [airflow] codecov-io edited a comment on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-605517449
 
 
   # [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=h1) Report
   > Merging [#7832](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=desc) into [master](https://codecov.io/gh/apache/airflow/commit/c232563111e47a7ef5b2d1808fa23d07d1cf13c0&el=desc) will **increase** coverage by `0.64%`.
   > The diff coverage is `92.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/airflow/pull/7832/graphs/tree.svg?width=650&height=150&src=pr&token=WdLKlKHOAU)](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #7832      +/-   ##
   ==========================================
   + Coverage   86.52%   87.17%   +0.64%     
   ==========================================
     Files         933      933              
     Lines       45312    45342      +30     
   ==========================================
   + Hits        39207    39526     +319     
   + Misses       6105     5816     -289     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [airflow/hooks/base\_hook.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9ob29rcy9iYXNlX2hvb2sucHk=) | `95.83% <ø> (ø)` | |
   | [airflow/models/dag.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9tb2RlbHMvZGFnLnB5) | `91.68% <ø> (ø)` | |
   | [airflow/providers/google/cloud/hooks/gcs.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL2Nsb3VkL2hvb2tzL2djcy5weQ==) | `86.43% <ø> (ø)` | |
   | [airflow/providers/grpc/hooks/grpc.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ3JwYy9ob29rcy9ncnBjLnB5) | `92.06% <ø> (ø)` | |
   | [airflow/www/views.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdmlld3MucHk=) | `76.46% <50.00%> (ø)` | |
   | [airflow/utils/code\_utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy91dGlscy9jb2RlX3V0aWxzLnB5) | `62.50% <62.50%> (ø)` | |
   | [airflow/cli/cli\_parser.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9jbGkvY2xpX3BhcnNlci5weQ==) | `97.24% <63.63%> (ø)` | |
   | [airflow/www/utils.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy93d3cvdXRpbHMucHk=) | `81.86% <85.71%> (ø)` | |
   | [airflow/operators/check\_operator.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9vcGVyYXRvcnMvY2hlY2tfb3BlcmF0b3IucHk=) | `93.25% <96.66%> (+0.77%)` | :arrow_up: |
   | [...s/google/marketing\_platform/operators/analytics.py](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree#diff-YWlyZmxvdy9wcm92aWRlcnMvZ29vZ2xlL21hcmtldGluZ19wbGF0Zm9ybS9vcGVyYXRvcnMvYW5hbHl0aWNzLnB5) | `99.18% <98.79%> (ø)` | |
   | ... and [35 more](https://codecov.io/gh/apache/airflow/pull/7832/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=footer). Last update [8b77787...3b336c6](https://codecov.io/gh/apache/airflow/pull/7832?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on issue #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#issuecomment-606808500
 
 
   @ashb - are you OK with the explanations? I'd love to merge it and do some follow-ups (and cherry-pick it to 1.10.10).

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


With regards,
Apache Git Services

[GitHub] [airflow] potiuk commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r401511649
 
 

 ##########
 File path: entrypoint.sh
 ##########
 @@ -0,0 +1,122 @@
+#!/usr/bin/env bash
+# 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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=10}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=2}
+
+    export DETECTED_DB_BACKEND=""
+    export DETECTED_DB_USER=""
+    export DETECTED_DB_HOST=""
+    export DETECTED_DB_PORT=""
+    export DETECTED_DB_SCHEMA=""
+    export DETECTED_DB_PARAMS=""
 
 Review comment:
   True 

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402402739
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,430 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
+
+The images are build with default extras - different extras for CI and production image and you
+can change the extras via the ``--extras`` parameters. You can see default extras used via
+``./breeze flags``.
+
+For example if you want to build python 3.7 version of production image with
+"all" extras installed you should run this command:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras "all" --production-image
+
+The command that builds the CI image is optimized to minimize the time needed to rebuild the image when
+the source code of Airflow evolves. This means that if you already have the image locally downloaded and
+built, the scripts will determine whether the rebuild is needed in the first place. Then the scripts will
+make sure that minimal number of steps are executed to rebuild parts of the image (for example,
+PIP dependencies) and will give you an image consistent with the one used during Continuous Integration.
+
+The command that builds the production image is optimised for size of the image.
+
+In Breeze by default, the airflow is installed using local sources of Apache Airflow.
+
+Yyou can also build production images from installed packages
+via providing ``--install-airflow-version`` parameter to Breeze:
+
+.. code-block::
+
+  ./breeze build-image --python 3.7 --extras=gcp --production-image --install-airflow-version=1.10.9
+
+This will build the image using command similar to:
+
+.. code-block::
+
+    pip install apache-airflow[gcp]==1.10.9 \
+       --constraint https://raw.githubusercontent.com/apache/airflow/v1-10-test/requirements/requirements-python3.7.txt
+
+This will also download entrypoint script from https://raw.githubusercontent.com/apache/airflow/v1-10-test/entrypoint.sh
+url. It is important so that we have matching version of the requirements.
+
+The requirement files and entrypoint only appeared in version 1.10.10 of airflow so if you install
+an earliier version -  both constraint and requirements should point to 1.10.10 version.
+
+Yyou can also build production images from specific Git verion
+via providing ``--install-airflow-reference`` parameter to Breeze:
+
+.. code-block::
+
+    pip install https://github.com/apache/airflow/archive/<tag>>.tar.gz#egg=apache-airflow \
+       --constraint https://raw.githubusercontent.com/apache/airflow/<tag>/requirements/requirements-python3.7.txt
+
+This will also Download entrypoint script from https://raw.githubusercontent.com/apache/airflow/<tag>>/entrypoint.sh
+url. It is important so that we have matching version of the requirements. Similar comment applies
+to 1.10.10 version as in case of v
+
+Technical details of Airflow images
+===================================
+
+The CI image is used by Breeze as shell image but it is also used during CI builds on Travis.
+The image is single segment image that contains Airflow installation with "all" dependencies installed.
+It is optimised for rebuild speed (AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD flag set to "true").
+It installs PIP dependencies from the current branch first - so that any changes in setup.py do not trigger
+reinstalling of all dependencies. There is a second step of installation that re-installs the dependencies
+from the latest sources so that we are sure that latest dependencies are installed.
+
+The production image is a multi-segment image. The first segment "airflow-build-image" contains all the
+build essentials and related dependencies that allow to install airflow locally. By default the image is
+build from a released version of Airflow from Github, but by providing some extra arguments you can also
+build it from local sources. This is particularly useful in CI environment where we are using the image
+to run Kubernetes tests. See below for the list of arguments that should be provided to build
+production image from the local sources.
+
+Note! Breeze by default builds production image from local sources. You can change it's behaviour by
+providing ``--install-airflow-version`` parameter, where you can specify the
+tag/branch used to download Airflow package from in github repository. You can
+also change the repository itself by adding --dockerhub-user and --dockerhub-repo flag values.
+
+Manually building the images
+----------------------------
+
+You can build the default production image with standard ``docker build`` command but they will only build
+default versions of the image and will not use the dockerhub versions of images as cache.
+
+
+CI images
+.........
+
+The following build arguments (``--build-arg`` in docker build command) can be used for CI images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``DEPENDENCIES_EPOCH_NUMBER``            | ``2``                                    | increasing this number will reinstall    |
+|                                          |                                          | all apt dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KUBECTL_VERSION``                      | ``v1.15.3``                              | version of kubectl installed             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``KIND_VERSION``                         | ``v0.6.1``                               | version of kind installed                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_NO_CACHE_DIR``                     | ``true``                                 | if true, then no pip cache will be       |
+|                                          |                                          | stored                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``HOME``                                 | ``/root``                                | Home directory of the root user (CI      |
+|                                          |                                          | image has root user as default)          |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/root/airflow``                        | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_SOURCES``                      | ``/opt/airflow``                         | Mounted sources of Airflow               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_DEPENDENCIES_EPOCH_NUMBER``        | ``3``                                    | increasing that number will reinstall    |
+|                                          |                                          | all PIP dependencies                     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_NO_CYTHON``                | ``1``                                    | if set to 1 no CYTHON compilation is     |
+|                                          |                                          | done for cassandra driver (much faster)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD`` | ``true``                                 | if set then PIP dependencies are         |
+|                                          |                                          | installed from repo first before they    |
+|                                          |                                          | are reinstalled from local sources. This |
+|                                          |                                          | allows for incremental faster builds     |
+|                                          |                                          | when requirements change                 |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``apache/airflow``                       | the repository from which PIP            |
+|                                          |                                          | dependencies are installed (CI           |
+|                                          |                                          | optimised)                               |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_BRANCH``                       | ``master``                               | the branch from which PIP dependencies   |
+|                                          |                                          | are installed (CI optimised)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_CI_BUILD_EPOCH``               | ``1``                                    | increasing this value will reinstall PIP |
+|                                          |                                          | dependencies from the repository from    |
+|                                          |                                          | scratch                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | ``all``                                  | extras to install                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``ADDITIONAL_PYTHON_DEPS``               | \```\`                                   | additional python dependencies to        |
+|                                          |                                          | install                                  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+Here are some examples of how CI images can built manually. CI is always built from local sources.
+
+This builds the CI image in version 3.7 with default extras ("all").
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7
+
+
+This builds the CI image in version 3.6 with "gcp" extra only.
+
+.. code-block::
+
+  docker build . -f Dockerfile.ci --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.6 --build-arg AIRFLOW_EXTRAS=gcp
+
+
+Production images
+.................
+
+The following build arguments (``--build-arg`` in docker build command) can be used for production images:
+
++------------------------------------------+------------------------------------------+------------------------------------------+
+| Build argument                           | Default value                            | Description                              |
++==========================================+==========================================+==========================================+
+| ``PYTHON_BASE_IMAGE``                    | ``python:3.6-slim-buster``               | Base python image                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PYTHON_MAJOR_MINOR_VERSION``           | ``3.6``                                  | major/minor version of Python (should    |
+|                                          |                                          | match base image)                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_VERSION``                      | ``2.0.0.dev0``                           | version of Airflow                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_ORG``                          | ``apache``                               | Github organisation from which Airflow   |
+|                                          |                                          | is installed (when installed from repo)  |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_REPO``                         | ``airflow``                              | Github repository from which Airflow is  |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GIT_REFERENCE``                | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which Airflow is         |
+|                                          |                                          | installed (when installed from repo)     |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``REQUIREMENTS_GIT_REFERENCE``           | ``master``                               | reference (branch or tag) from Github    |
+|                                          |                                          | repository from which requirements are   |
+|                                          |                                          | downloaded for constraints (when         |
+|                                          |                                          | installed from repo).                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``WWW_FOLDER``                           | ``www``                                  | folder where www pages are generated -   |
+|                                          |                                          | should be set to www_rbac in case of     |
+|                                          |                                          | 1.10 image builds.                       |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_EXTRAS``                       | (see Dockerfile)                         | Default extras with which airflow is     |
+|                                          |                                          | installed                                |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_HOME``                         | ``/opt/airflow``                         | Airflow’s HOME (that’s where logs and    |
+|                                          |                                          | sqlite databases are stored)             |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_UID``                          | ``50000``                                | Airflow user UID                         |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``AIRFLOW_GID``                          | ``50000``                                | Airflow group GID                        |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``PIP_VERSION``                          | ``19.0.2``                               | version of PIP to use                    |
++------------------------------------------+------------------------------------------+------------------------------------------+
+| ``CASS_DRIVER_BUILD_CONCURRENCY``        | ``8``                                    | Number of processors to use for          |
+|                                          |                                          | cassandra PIP install (speeds up         |
+|                                          |                                          | installing in case cassandra extra is    |
+|                                          |                                          | used).                                   |
++------------------------------------------+------------------------------------------+------------------------------------------+
+
+There are build arguments that determine the installation mechanism of Apache Airflow for the
+production image. There are three types of build:
+
+* From local sources (by default for example when you use ``docker build .``)
+* You can build the image from released PyPi airflow package (used to build the official Docker image)
+* You can build the image from any version in GitHub repository(this is used mostly for system testing).
+
++-----------------------------------+-----------------------------------+
+| Build argument                    | What to specify                   |
++===================================+===================================+
+| ``AIRFLOW_INSTALL_SOURCES``       | Should point to the sources of    |
+|                                   | of Apache Airflow. It can be      |
+|                                   | either "." for installation from  |
+|                                   | local sources, "apache-airflow"   |
+|                                   | for installation from packages    |
+|                                   | and URL to installation from      |
+|                                   | GitHub repository (see below)     |
+|                                   | to install from any GitHub        |
+|                                   | version                           |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_INSTALL_VERSION``       | Optional - might be used for      |
+|                                   | package installation case to      |
+|                                   | set Airflow version for example   |
+|                                   | "==1.10.10"                       |
++-----------------------------------+-----------------------------------+
+| ``CONSTRAINT_REQUIREMENTS``       | Should point to requirements file |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``ENTRYPOINT_FILE``               | Should point to entrypoint.sh     |
+|                                   | file in case of installation from |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_WWW``                   | In case of Airflow 2.0 it should  |
+|                                   | be "www", in case of Airflow 1.10 |
+|                                   | series it should be "www_rbac".   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_FROM``          | Sources of Airflow. Should be set |
+|                                   | to "Dockerfile" to avoid costly   |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+| ``AIRFLOW_SOURCES_TO``            | Target for Airflow sources. Set   |
+|                                   | to "/Dockerfile" to avoid costly  |
+|                                   | Docker context copying            |
+|                                   | in case of installation from      |
+|                                   | the package or from GitHub URL.   |
+|                                   | See examples below                |
++-----------------------------------+-----------------------------------+
+
+
+This builds production image in version 3.6 with default extras from the local sources:
+
+.. code-block::
+
+  docker build .
+
+This builds the production image in version 3.7 with default extras from 1.10.9 tag and
+requirements taken from v1-10-test branch in Github.
+Note that versions 1.10.9 and below have no requirements so requirements should be taken from head of
+the v1-10-test branch. Once we release 1.10.10 we can take them from the 1.10.10 tag. Also
+Note that in case of Airflow 1.10 we need to specify "www_rbac" instead of "wwww" for
+WWW_FOLDER argument.
+
+.. code-block::
+
+  docker build . \
+    --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 \
+    --build-arg AIRFLOW_INSTALL_SOURCES="https://github.com/apache/airflow/archive/1.10.10.tar.gz#egg=apache-airflow" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
+    --build-arg WWW_FOLDER="www_rbac"
+
+This builds the production image in version 3.6 with default extras from current sources.
+
+.. code-block::
+
+  docker build . --build-arg PYTHON_BASE_IMAGE="python:3.7-slim-buster" \
+    --build-arg PYTHON_MAJOR_MINOR_VERSION=3.7 --build-arg COPY_SOURCE=. \
+    --build-arg COPY_TARGET=/opt/airflow --build-arg AIRFLOW_SOURCES=/opt/airflow \
+    --build-arg CONSTRAINT_REQUIREMENTS=requirements/requirements-python3.7.txt" \
+    --build-arg ENTRYPOINT_FILE=entrypoint.sh \
+    --build-arg AIRFLOW_INSTALL_SOURCES="apache-airflow" \
+    --build-arg AIRFLOW_INSTALL_VERSION="==1.10.10" \
+    --build-arg CONSTRAINT_REQUIREMENTS="https://raw.githubusercontent.com/apache/airflow/1.10.10/requirements/requirements-python3.7.txt"
+    --build-arg ENTRYPOINT_FILE="https://raw.githubusercontent.com/apache/airflow/1.10.10/entrypoint.sh" \
+    --build-arg SOURCES_FROM="Dockerfile" \
+    --build-arg SOURCES_TO="/Dockerfile" \
+    --build-arg WWW_FOLDER="www_rbac"
+
+Image manifests
+---------------
+
+Together with the main CI images we also build and push image manifests. Those manifests are very small images
+that contain only results of the docker inspect for the image. This is in order to be able to
+determine very quickly if the image in the docker registry has changed a lot since the last time.
+Unfortunately docker registry (specifically dockerhub registry) has no anonymous way of querying image
+details via API, you need to download the image to inspect it. We overcame it in the way that
+always when we build the image we build a very small image manifest and push it to registry together
+with the main CI image. The tag for the manifest image is the same as for the image it refers
+to with added ``-manifest`` suffix. The manifest image for ``apache/airflow:master-python3.6-ci`` is named
+``apache/airflow:master-python3.6-ci-manifest``.
+
+
+Pulling the Latest Images
+-------------------------
+
+Sometimes the image needs to be rebuilt from scratch. This is required, for example,
+when there is a security update of the Python version that all the images are based on and new version
+of the image is pushed to the repository. In this case it is usually faster to pull the latest
+images rather than rebuild them from scratch.
+
+You can do it via the ``--force-pull-images`` flag to force pulling the latest images from the Docker Hub.
+
+For production image:
+
+.. code-block::
+
+  ./breeze build-image --force-pull-images --production-image
+
+For CI image Breeze automatically uses force pulling in case it determines that your image is very outdated,
+however uou can also force it with the same flag.
+
+.. code-block::
+
+  ./breeze build-image --force-pull-images
+
+Using the images
+================
+
+Both images have entrypoint set as dumb-init with entrypoint.sh script executed (in order to forward
+signals). This entrypoint works as follows:
+
+* If ``AIRFLOW__CORE__SQL_ALCHEMY_CONN`` variable is passed to the container and it is either mysql or postgres
+  SQL alchemy connection, then the connection is checked and the script waits until the database is reachable.
+* If no ``AIRFLOW__CORE__SQL_ALCHEMY_CONN`` variable is set or if it is set to sqlite SQL alchemy connection
+  then db reset is executed.
+* If ``AIRFLOW__CELERY__BROKER_URL`` variable is passed and scheduler, worker of flower command is used then
+  the connection is checked and the script waits until the Celery broker database is reachable.
+* If ``DB_RESET`` is specified - the 'airflow db reset' is used.
+
+* If no argument is specified - you are dropped in bash shell.
+* If there are any arguments they are passed to "airflow" command
 
 Review comment:
   A thought: should we add this or shorter info to docs? In this way, information about docker image will appear on Airflow website.

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


With regards,
Apache Git Services

[GitHub] [airflow] turbaszek commented on a change in pull request #7832: Add production image support

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #7832: Add production image support
URL: https://github.com/apache/airflow/pull/7832#discussion_r402398627
 
 

 ##########
 File path: IMAGES.rst
 ##########
 @@ -0,0 +1,430 @@
+ .. 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.
+
+.. contents:: :local:
+
+Airflow docker images
+=====================
+
+Airflow has two images (build from Dockerfiles):
+
+* CI image (Dockerfile.ci) - used for running tests and local development
+* Production image (Dockerfile) - used to run production-ready Airflow installations
+
+Image naming conventions
+========================
+
+The images are named as follows:
+
+``apache/airflow:<BRANCH_OR_TAG>-python<PYTHON_MAJOR_MINOR_VERSION>[-ci][-manifest]``
+
+where:
+
+* BRANCH_OR_TAG - branch or tag used when creating the image. Examples: master, v1-10-test, 1.10.10
+  The ``master`` and ``v1-10-test`` labels are built from branches so they change over time. the 1.10.* and in
+  the future ``2.*`` labels are build from git tags and they are "fixed" once built.
+* PYTHON_MAJOR_MINOR_VERSION - version of python used to build the image. Examples: 3.5, 3.7
+* The ``-ci`` suffix is added for CI images
+* The ``-manifest`` is added for manifest images (see below for explanation of manifest images)
+
+Building docker images
+======================
+
+The easiest way to build those images is to use `<BREEZE.rst>`_.
+
+You can build the CI image using this command:
+
+.. code-block::
+
+  ./breeze build-image
+
+You can build production image using this command:
+
+.. code-block::
+
+  ./breeze build-image --production-image
+
+By adding ``--python <PYTHON_MAJOR_MINOR_VERSION>`` parameter you can build the
+image version for the chosen python version.
 
 Review comment:
   Are there any limitations for the version?

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


With regards,
Apache Git Services