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 2021/12/08 01:29:12 UTC

[GitHub] [airflow] dstandish commented on a change in pull request #19729: Local kubernetes executor

dstandish commented on a change in pull request #19729:
URL: https://github.com/apache/airflow/pull/19729#discussion_r764477823



##########
File path: airflow/executors/local_kubernetes_executor.py
##########
@@ -0,0 +1,204 @@
+#
+# 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.
+from typing import Dict, List, Optional, Set, Union
+
+from airflow.configuration import conf
+from airflow.executors.base_executor import CommandType, EventBufferValueType, QueuedTaskInstanceType
+from airflow.executors.kubernetes_executor import KubernetesExecutor
+from airflow.executors.local_executor import LocalExecutor
+from airflow.models.taskinstance import SimpleTaskInstance, TaskInstance, TaskInstanceKey
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class LocalKubernetesExecutor(LoggingMixin):
+    """
+    LocalKubernetesExecutor consists of LocalExecutor and KubernetesExecutor.
+    It chooses an executor to use based on the queue defined on the task.
+    When the queue is the value of ``kubernetes_queue`` in section ``[local_kubernetes_executor]``

Review comment:
       ```suggestion
       When the task's queue is the value of ``kubernetes_queue`` in section ``[local_kubernetes_executor]``
   ```

##########
File path: airflow/executors/local_kubernetes_executor.py
##########
@@ -0,0 +1,204 @@
+#
+# 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.
+from typing import Dict, List, Optional, Set, Union
+
+from airflow.configuration import conf
+from airflow.executors.base_executor import CommandType, EventBufferValueType, QueuedTaskInstanceType
+from airflow.executors.kubernetes_executor import KubernetesExecutor
+from airflow.executors.local_executor import LocalExecutor
+from airflow.models.taskinstance import SimpleTaskInstance, TaskInstance, TaskInstanceKey
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class LocalKubernetesExecutor(LoggingMixin):
+    """
+    LocalKubernetesExecutor consists of LocalExecutor and KubernetesExecutor.
+    It chooses an executor to use based on the queue defined on the task.

Review comment:
       ```suggestion
       It chooses the executor to use based on the queue defined on the task.
   ```

##########
File path: docs/apache-airflow/executor/local_kubernetes.rst
##########
@@ -0,0 +1,30 @@
+ .. 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.
+
+
+.. _executor:LocalKubernetesExecutor:
+
+LocalKubernetes Executor
+=========================
+
+The :class:`~airflow.executors.local_kubernetes_executor.LocalKubernetesExecutor` allows users
+to simultaneously run a ``LocalExecutor`` and a ``KubernetesExecutor``.
+An executor is chosen to run a task based on the task's queue.
+
+``LocalKubernetesExecutor`` provides the capability of running low-intensive tasks locally
+using ``LocalExecutor`` and to handle the high load at the peak time and runtime isolation
+of the ``KubernetesExecutor``.

Review comment:
       ```suggestion
   ``LocalKubernetesExecutor`` provides the capability of running tasks with either ``LocalExecutor``,
   which runs tasks within the scheduler service, or with ``KubernetesExecutor``, which runs each task
   in its own pod on a kubernetes cluster.
   ```
   

##########
File path: airflow/executors/local_kubernetes_executor.py
##########
@@ -0,0 +1,204 @@
+#
+# 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.
+from typing import Dict, List, Optional, Set, Union
+
+from airflow.configuration import conf
+from airflow.executors.base_executor import CommandType, EventBufferValueType, QueuedTaskInstanceType
+from airflow.executors.kubernetes_executor import KubernetesExecutor
+from airflow.executors.local_executor import LocalExecutor
+from airflow.models.taskinstance import SimpleTaskInstance, TaskInstance, TaskInstanceKey
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class LocalKubernetesExecutor(LoggingMixin):
+    """
+    LocalKubernetesExecutor consists of LocalExecutor and KubernetesExecutor.
+    It chooses an executor to use based on the queue defined on the task.
+    When the queue is the value of ``kubernetes_queue`` in section ``[local_kubernetes_executor]``
+    of the configuration (default value: `kubernetes`), KubernetesExecutor is selected to run the task,
+    otherwise, LocalExecutor is used.
+    """
+
+    supports_ad_hoc_ti_run: bool = True

Review comment:
       i think you need also to update the `flash` message [here](https://github.com/apache/airflow/blob/b5c0158b2eb646eb1db5d2c094d3da8f88a08a8b/airflow/www/views.py#L1607-L1609)

##########
File path: airflow/executors/local_kubernetes_executor.py
##########
@@ -0,0 +1,204 @@
+#
+# 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.
+from typing import Dict, List, Optional, Set, Union
+
+from airflow.configuration import conf
+from airflow.executors.base_executor import CommandType, EventBufferValueType, QueuedTaskInstanceType
+from airflow.executors.kubernetes_executor import KubernetesExecutor
+from airflow.executors.local_executor import LocalExecutor
+from airflow.models.taskinstance import SimpleTaskInstance, TaskInstance, TaskInstanceKey
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class LocalKubernetesExecutor(LoggingMixin):
+    """
+    LocalKubernetesExecutor consists of LocalExecutor and KubernetesExecutor.
+    It chooses an executor to use based on the queue defined on the task.
+    When the queue is the value of ``kubernetes_queue`` in section ``[local_kubernetes_executor]``
+    of the configuration (default value: `kubernetes`), KubernetesExecutor is selected to run the task,
+    otherwise, LocalExecutor is used.
+    """
+
+    supports_ad_hoc_ti_run: bool = True

Review comment:
       on a related note... according to [that message](https://github.com/apache/airflow/blob/b5c0158b2eb646eb1db5d2c094d3da8f88a08a8b/airflow/www/views.py#L1607-L1609), celery / CKE, and KE all support "ad hoc" run.  why doesn't LocalExecutor? it looks like it's because it starts up an instance of the executor from within the webserver.  i'm surprised this works with k8s because i thought i recalled some issues where the webserver didn't have the right roles to create pods...  but that's another story.   Anyway, perhaps we need to add some way, more sophisticated now than `supports_ad_hoc_ti_run`, such that the ad hoc trigger  will work for tasks with k8s queue but not for local.  thoughts @kaxil ?




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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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