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/01 10:17:07 UTC

[GitHub] [airflow] andreaspeters commented on a change in pull request #10977: Reinclude/Add a new Mesos Executor

andreaspeters commented on a change in pull request #10977:
URL: https://github.com/apache/airflow/pull/10977#discussion_r533277946



##########
File path: airflow/executors/mesos_executor.py
##########
@@ -0,0 +1,422 @@
+#
+# 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.
+
+import threading
+from queue import Queue
+from typing import Any, Dict, Optional
+
+from avmesos.client import MesosClient
+
+from airflow.configuration import conf
+from airflow.exceptions import AirflowException
+from airflow.executors.base_executor import BaseExecutor, CommandType
+from airflow.models.taskinstance import TaskInstanceKey
+from airflow.utils.session import provide_session
+from airflow.utils.state import State
+
+DEFAULT_FRAMEWORK_NAME = 'Airflow'
+FRAMEWORK_CONNID_PREFIX = 'mesos_framework_'
+
+
+def get_framework_name():
+    """Get the mesos framework name if its set in airflow.cfg"""
+    if not conf.get('mesos', 'FRAMEWORK_NAME'):
+        return DEFAULT_FRAMEWORK_NAME
+    return conf.get('mesos', 'FRAMEWORK_NAME')
+
+
+# pylint: disable=too-many-nested-blocks
+# pylint: disable=too-many-instance-attributes
+class AirflowMesosScheduler(MesosClient):
+    """
+    Airflow Mesos scheduler implements mesos scheduler interface
+    to schedule airflow tasks on mesos
+    Basically, it schedules a command like
+    'airflow run <dag_id> <task_instance_id> <start_date> --local -p=<pickle>'
+    to run on a mesos slave
+    """
+
+    # pylint: disable=super-init-not-called
+    def __init__(self, executor, task_queue, result_queue, task_cpu: int = 1, task_mem: int = 256):
+        self.task_queue = task_queue
+        self.result_queue = result_queue
+        self.task_cpu = task_cpu
+        self.task_mem = task_mem
+        self.task_counter = 0
+        self.task_key_map: Dict[str, str] = {}
+        self.log = executor.log
+        self.client = executor.client
+        self.executor = executor
+        self.driver = None
+
+        if not conf.get('mesos', 'DOCKER_IMAGE_SLAVE'):
+            self.log.error("Expecting docker image for  mesos executor")
+            raise AirflowException("mesos.slave_docker_image not provided for mesos executor")
+
+        if not conf.get('mesos', 'DOCKER_VOLUME_DRIVER'):
+            self.log.error("Expecting docker volume driver for mesos executor")
+            raise AirflowException("mesos.docker_volume_driver not provided for mesos executor")
+
+        if not conf.get('mesos', 'DOCKER_VOLUME_DAG_NAME'):
+            self.log.error("Expecting docker volume dag name for mesos executor")
+            raise AirflowException("mesos.docker_volume_dag_name not provided for mesos executor")
+
+        if not conf.get('mesos', 'DOCKER_VOLUME_DAG_CONTAINER_PATH'):
+            self.log.error("Expecting docker volume dag container path for mesos executor")
+            raise AirflowException("mesos.docker_volume_dag_container_path not provided for mesos executor")
+
+        if not conf.get('mesos', 'DOCKER_VOLUME_LOGS_NAME'):
+            self.log.error("Expecting docker volume logs name for mesos executor")
+            raise AirflowException("mesos.docker_volume_logs_name not provided for mesos executor")
+
+        if not conf.get('mesos', 'DOCKER_VOLUME_LOGS_CONTAINER_PATH'):
+            self.log.error("Expecting docker volume logs container path for mesos executor")
+            raise AirflowException("mesos.docker_volume_logs_container_path not provided for mesos executor")
+
+        if not conf.get('mesos', 'DOCKER_SOCK'):
+            self.log.error("Expecting docker sock path for mesos executor")
+            raise AirflowException("mesos.docker_sock not provided for mesos executor")
+
+        self.mesos_slave_docker_image = conf.get('mesos', 'DOCKER_IMAGE_SLAVE').replace('"', '')
+        self.mesos_docker_volume_driver = conf.get('mesos', 'DOCKER_VOLUME_DRIVER').replace('"', '')
+        self.mesos_docker_volume_dag_name = conf.get('mesos', 'DOCKER_VOLUME_DAG_NAME').replace('"', '')
+        self.mesos_docker_volume_dag_container_path = conf.get(
+            'mesos', 'DOCKER_VOLUME_DAG_CONTAINER_PATH'
+        ).replace('"', '')
+        self.mesos_docker_volume_logs_name = conf.get('mesos', 'DOCKER_VOLUME_LOGS_NAME').replace('"', '')
+        self.mesos_docker_volume_logs_container_path = conf.get(
+            'mesos', 'DOCKER_VOLUME_LOGS_CONTAINER_PATH'
+        ).replace('"', '')
+        self.mesos_docker_sock = conf.get('mesos', 'DOCKER_SOCK')
+        self.core_sql_alchemy_conn = conf.get('core', 'SQL_ALCHEMY_CONN')
+        self.core_fernet_key = conf.get('core', 'FERNET_KEY')
+
+    def resource_offers(self, offers):
+        """If we got a offer, run a queued task"""
+        self.log.debug('MESOS OFFER')
+        for i, offer in enumerate(offers):
+            if i == 0:
+                self.run_job(offer)
+            offer.decline()
+            i += 1
+
+    def run_job(self, mesos_offer):
+        """Start a queued Airflow task in Mesos"""
+        offer = mesos_offer.get_offer()
+        tasks = []
+        option = {}
+        offer_cpus = 0
+        offer_mem = 0
+        for resource in offer['resources']:
+            if resource['name'] == "cpus":
+                offer_cpus += resource['scalar']['value']
+            elif resource['name'] == "mem":
+                offer_mem += resource['scalar']['value']
+
+        self.log.debug(
+            "Received offer %s with cpus: %s and mem: %s", offer['id']['value'], offer_cpus, offer_mem
+        )
+
+        remaining_cpus = offer_cpus
+        remaining_mem = offer_mem
+
+        while (
+            (not self.task_queue.empty())
+            and remaining_cpus >= self.task_cpu
+            and remaining_mem >= self.task_mem
+        ):
+
+            key, cmd, executor_config = self.task_queue.get()
+            self.log.debug(executor_config)
+            tid = self.task_counter
+            self.task_counter += 1
+            self.task_key_map[str(tid)] = key
+
+            port_begin = 31000 + tid
+            port_end = 31000 + tid
+
+            self.log.debug("Launching task %d using offer %s", tid, offer['id']['value'])
+
+            task = {
+                'name': "AirflowTask %d" % tid,
+                'task_id': {'value': str(tid)},
+                'agent_id': {'value': offer['agent_id']['value']},
+                'resources': [
+                    {'name': 'cpus', 'type': 'SCALAR', 'scalar': {'value': self.task_cpu}},
+                    {'name': 'mem', 'type': 'SCALAR', 'scalar': {'value': self.task_mem}},
+                    {
+                        'name': 'ports',
+                        'type': 'RANGES',
+                        'ranges': {'range': [{'begin': port_begin, 'end': port_end}]},
+                    },
+                ],
+                'command': {
+                    'shell': 'true',
+                    'environment': {
+                        'variables': [
+                            {'name': 'AIRFLOW__CORE__SQL_ALCHEMY_CONN', 'value': self.core_sql_alchemy_conn},
+                            {'name': 'AIRFLOW__CORE__FERNET_KEY', 'value': self.core_fernet_key},
+                            {'name': 'AIRFLOW__CORE__LOGGING_LEVEL', 'value': 'DEBUG'},
+                        ]
+                    },
+                    'value': " ".join(cmd),
+                },
+                'container': {
+                    'type': 'DOCKER',
+                    'volumes': [
+                        {
+                            'container_path': self.mesos_docker_volume_dag_container_path,
+                            'mode': 'RO',
+                            'source': {
+                                'type': 'DOCKER_VOLUME',
+                                'docker_volume': {
+                                    'driver': self.mesos_docker_volume_driver,
+                                    'name': self.mesos_docker_volume_dag_name,
+                                },
+                            },
+                        },
+                        {
+                            'container_path': self.mesos_docker_volume_logs_container_path,
+                            'mode': 'RW',
+                            'source': {
+                                'type': 'DOCKER_VOLUME',
+                                'docker_volume': {
+                                    'driver': self.mesos_docker_volume_driver,
+                                    'name': self.mesos_docker_volume_logs_name,
+                                },
+                            },
+                        },
+                    ],
+                    'docker': {
+                        'image': self.mesos_slave_docker_image,
+                        'force_pull_image': 'true',
+                        'privileged': 'true',
+                        'parameters': [
+                            {'key': 'volume', 'value': self.mesos_docker_sock + ':/var/run/docker.sock'}

Review comment:
       I remove 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