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

[GitHub] [airflow] dimberman opened a new pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

dimberman opened a new pull request #12766:
URL: https://github.com/apache/airflow/pull/12766


   Fixes issue on openshift caused by KubernetesExecutor pods not running
   via the entrypoint script
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   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).
   


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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535093360



##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -129,7 +129,7 @@ if [[ ${AIRFLOW_COMMAND} == "bash" ]]; then
 elif [[ ${AIRFLOW_COMMAND} == "python" ]]; then
    shift
    exec "python" "${@}"
+elif [[ ${AIRFLOW_COMMAND} == "airflow" ]]; then
+   shift
+   exec "airflow" "${@}"
 fi
-
-# Run the command
-exec airflow "${@}"

Review comment:
       Line 115 is now not going to check the right thing anymore.




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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535094089



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       This needs a note in the 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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535510419



##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -129,7 +129,7 @@ if [[ ${AIRFLOW_COMMAND} == "bash" ]]; then
 elif [[ ${AIRFLOW_COMMAND} == "python" ]]; then
    shift
    exec "python" "${@}"
+elif [[ ${AIRFLOW_COMMAND} == "airflow" ]]; then
+   shift
+   exec "airflow" "${@}"
 fi
-
-# Run the command
-exec airflow "${@}"

Review comment:
       I think it needs to support both modes - this will break the chart otherwise!




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



[GitHub] [airflow] ashb commented on pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#issuecomment-739907737


   Please rebase onto mainline once #12878 has been merged.


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



[GitHub] [airflow] github-actions[bot] commented on pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#issuecomment-739314539


   [The Workflow run](https://github.com/apache/airflow/actions/runs/402737123) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535381520



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       Do we need to mark this if it's not a publicly advertised feature?




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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535380801



##########
File path: airflow/kubernetes/pod_generator.py
##########
@@ -351,7 +351,7 @@ def construct_pod(  # pylint: disable=too-many-arguments
         try_number: int,
         kube_image: str,
         date: datetime.datetime,
-        command: List[str],
+        args: List[str],

Review comment:
       The PodGenerator is only used internally by the KubernetesExecutor, we don't discuss it anywhere in the airflow documentation. It's not advertised as a part of the Airflow API.




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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r536271178



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       True.
   
   So 2.0 is the time to do it :D 




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



[GitHub] [airflow] github-actions[bot] commented on pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#issuecomment-739662058


   [The Workflow run](https://github.com/apache/airflow/actions/runs/405022738) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535093923



##########
File path: airflow/kubernetes/pod_generator.py
##########
@@ -351,7 +351,7 @@ def construct_pod(  # pylint: disable=too-many-arguments
         try_number: int,
         kube_image: str,
         date: datetime.datetime,
-        command: List[str],
+        args: List[str],

Review comment:
       I think pod generator should have options for command and 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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535784388



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       Ohhh ok I see what you're saying. This affects a user who would say... modify the command and args section of the pod in the pod_mutation_hook.




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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535509845



##########
File path: airflow/kubernetes/pod_generator.py
##########
@@ -351,7 +351,7 @@ def construct_pod(  # pylint: disable=too-many-arguments
         try_number: int,
         kube_image: str,
         date: datetime.datetime,
-        command: List[str],
+        args: List[str],

Review comment:
       Oh, it's not used in mutation hook anymore. Cool, this is fine then




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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535414965



##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -129,7 +129,7 @@ if [[ ${AIRFLOW_COMMAND} == "bash" ]]; then
 elif [[ ${AIRFLOW_COMMAND} == "python" ]]; then
    shift
    exec "python" "${@}"
+elif [[ ${AIRFLOW_COMMAND} == "airflow" ]]; then
+   shift
+   exec "airflow" "${@}"
 fi
-
-# Run the command
-exec airflow "${@}"

Review comment:
       @ashb so would the idea be that we just run the command as given? 




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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r534550581



##########
File path: chart/templates/webserver/webserver-deployment.yaml
##########
@@ -94,7 +94,7 @@ spec:
         - name: webserver
           image: {{ template "airflow_image" . }}
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
-          args: ["webserver"]
+          args: ["airflow", "webserver"]

Review comment:
       https://github.com/astronomer/airflow/blob/fix-entrypoint-for-openshift/chart/templates/workers/worker-deployment.yaml#L118




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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r534550285



##########
File path: chart/templates/webserver/webserver-deployment.yaml
##########
@@ -94,7 +94,7 @@ spec:
         - name: webserver
           image: {{ template "airflow_image" . }}
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
-          args: ["webserver"]
+          args: ["airflow", "webserver"]

Review comment:
       @turbaszek the worker deployments already use airflow commands




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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r536346427



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       @ashb fair enough :). I'll add this, add to the UPDATING, and then add an upgrade rule after.




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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r537646057



##########
File path: UPGRADING_TO_2.0.md
##########
@@ -493,6 +494,27 @@ second_task = PythonOperator(
 For Airflow 2.0, the traditional `executor_config` will continue operation with a deprecation warning,
 but will be removed in a future version.
 
+### The "airflow run" Command Will Row run Through the ``args`` Argument Instead of the ``command`` Argument
+
+To ensure that the KubernetesExecutor can work with entrypoint scripts (which are critical to running on OpenShift),
+KubernetesExecutor pods will run the ``airflow run`` command via the ``arg`` argument instead of the ``command``

Review comment:
       .md, not .rst -- single backticks for code 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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535510784



##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -129,7 +129,7 @@ if [[ ${AIRFLOW_COMMAND} == "bash" ]]; then
 elif [[ ${AIRFLOW_COMMAND} == "python" ]]; then
    shift
    exec "python" "${@}"
+elif [[ ${AIRFLOW_COMMAND} == "airflow" ]]; then
+   shift
+   exec "airflow" "${@}"
 fi
-
-# Run the command
-exec airflow "${@}"

Review comment:
       Oh you updated the 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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r537636414



##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -111,8 +111,7 @@ else
     verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
 fi
 
-# The Bash and python commands still should verify the basic connections so they are run after the
-# DB check but before the broker check
+

Review comment:
       Could you add/keep this comment.

##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -111,8 +111,7 @@ else
     verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
 fi
 
-# The Bash and python commands still should verify the basic connections so they are run after the
-# DB check but before the broker check
+

Review comment:
       Could you add back/keep this comment.




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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535784071



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       @ashb but no one relies on the old behavior. This is not a feature that people would use when building DAGs. It only works in the context of adding in stuff specific to the KubernetesExecutor.




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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r536334220



##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -111,25 +111,25 @@ else
     verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
 fi
 
-# Note: the broker backend configuration concerns only a subset of Airflow components
-if [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
-    if [[ -n "${AIRFLOW__CELERY__BROKER_URL_CMD=}" ]]; then
-        verify_db_connection "$(eval "$AIRFLOW__CELERY__BROKER_URL_CMD")"
-    else
-        AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
-        if [[ -n ${AIRFLOW__CELERY__BROKER_URL=} ]]; then
-            verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
-        fi
-    fi
-fi
 
 if [[ ${AIRFLOW_COMMAND} == "bash" ]]; then
    shift
    exec "/bin/bash" "${@}"
-elif [[ ${AIRFLOW_COMMAND} == "python" ]]; then
-   shift
-   exec "python" "${@}"
 elif [[ ${AIRFLOW_COMMAND} == "airflow" ]]; then
+   AIRFLOW_COMMAND="${2}"
+   # Note: the broker backend configuration concerns only a subset of Airflow components
+    if [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then

Review comment:
       Oh this should also have `celery` in the list for 2.0 compat (it's `airflow celery worker` there, `airflow worker` still on 1.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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r536269411



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       @ashb but this WILL break pod_mutation_hooks
   
   example:
   
   ```
   def pod_mutation_hook(pod):
       if "foo" in pod.spec.containers[0].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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r537645829



##########
File path: UPGRADING_TO_2.0.md
##########
@@ -493,6 +494,27 @@ second_task = PythonOperator(
 For Airflow 2.0, the traditional `executor_config` will continue operation with a deprecation warning,
 but will be removed in a future version.
 
+### The "airflow run" Command Will Row run Through the ``args`` Argument Instead of the ``command`` Argument

Review comment:
       ```suggestion
   ### The "airflow run" Command Will Row run through the `args` argument Instead of the `command` argument
   ```




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



[GitHub] [airflow] dimberman merged pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman merged pull request #12766:
URL: https://github.com/apache/airflow/pull/12766


   


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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535509408



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       It changes how the pod runs, so anyone relying on the old behaviour might get 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



[GitHub] [airflow] dimberman commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535784483



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       @ashb in that case do we really want to break 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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535092513



##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -129,7 +129,7 @@ if [[ ${AIRFLOW_COMMAND} == "bash" ]]; then
 elif [[ ${AIRFLOW_COMMAND} == "python" ]]; then
    shift
    exec "python" "${@}"
+elif [[ ${AIRFLOW_COMMAND} == "airflow" ]]; then
+   shift
+   exec "airflow" "${@}"
 fi
-
-# Run the command
-exec airflow "${@}"

Review comment:
       Could we replace this all with just `exec "${@}"`?




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



[GitHub] [airflow] turbaszek commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r535028363



##########
File path: chart/templates/webserver/webserver-deployment.yaml
##########
@@ -94,7 +94,7 @@ spec:
         - name: webserver
           image: {{ template "airflow_image" . }}
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
-          args: ["webserver"]
+          args: ["airflow", "webserver"]

Review comment:
       Oh indeed, 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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r537635892



##########
File path: scripts/in_container/prod/entrypoint_prod.sh
##########
@@ -125,8 +124,8 @@ elif [[ ${AIRFLOW_COMMAND} == "airflow" ]]; then
 fi
 
 # Note: the broker backend configuration concerns only a subset of Airflow components
-if [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
-    if [[ -n "${AIRFLOW__CELERY__BROKER_URL_CMD=}" ]]; then
+if [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|celery|worker|flower)$ ]]; then
+  if [[ -n "${AIRFLOW__CELERY__BROKER_URL_CMD=}" ]]; then

Review comment:
       ```suggestion
       if [[ -n "${AIRFLOW__CELERY__BROKER_URL_CMD=}" ]]; then
   ```




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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r536256990



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       It's not about pod mutation hook - but about _how the pod is run_.
   
   Previously this did the effect of `docker run --entrypoint airflow ... run`
   
   Now it runs the entrypoint (which is more correct) and needed to actually fix that bug, so yes, we want this, but just need to include a note about it in 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



[GitHub] [airflow] ashb commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r536354882



##########
File path: airflow/executors/kubernetes_executor.py
##########
@@ -297,7 +297,7 @@ def run_next(self, next_job: KubernetesJobType) -> None:
             kube_image=self.kube_config.kube_image,
             try_number=try_number,
             date=execution_date,
-            command=command,
+            args=command,

Review comment:
       Upgrade rule may be tricky for this one. Ace of you can do it, but we can't get a rule for everything




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



[GitHub] [airflow] turbaszek commented on a change in pull request #12766: Kubernetes worker pod doesn't use docker container entrypoint

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #12766:
URL: https://github.com/apache/airflow/pull/12766#discussion_r534525811



##########
File path: chart/templates/webserver/webserver-deployment.yaml
##########
@@ -94,7 +94,7 @@ spec:
         - name: webserver
           image: {{ template "airflow_image" . }}
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
-          args: ["webserver"]
+          args: ["airflow", "webserver"]

Review comment:
       Should it be also adjusted in workers deployment?




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