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/11/27 14:09:29 UTC

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

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



##########
File path: airflow/config_templates/default_airflow.cfg
##########
@@ -1014,3 +1014,67 @@ shards = 5
 
 # comma separated sensor classes support in smart_sensor.
 sensors_enabled = NamedHivePartitionSensor
+
+[mesos]
+# Mesos `master` address which MesosExecutor will connect to.
+master = localhost:5050
+
+# The framework name which Airflow scheduler will register itself as on mesos,
+framework_name = Airflow
+
+# Number of cpu cores required for running one task instance using
+# `airflow run <dag_id> <task_id> <execution_date> --local -p <pickle_id>`
+# command on a mesos slave,
+task_cpu = 1
+
+# Memory in MB required for running one task instance using
+# `airflow run <dag_id> <task_id> <execution_date> --local -p <pickle_id>`
+# command on a mesos slave.
+task_memory = 200
+
+# Enable framework checkpointing for mesos
+# See http://mesos.apache.org/documentation/latest/slave-recovery/
+checkpoint = False
+
+# Failover timeout in milliseconds.
+# When checkpointing is enabled and this option is set, Mesos waits
+# until the configured timeout for
+# the MesosExecutor framework to re-register after a failover. Mesos
+# shuts down running tasks if the
+# MesosExecutor framework fails to re-register within this timeframe.
+failover_timeout = 604800
+
+# Enable framework authentication for mesos
+# See http://mesos.apache.org/documentation/latest/configuration/
+authenticate = False
+
+# Mesos credentials, if authentication is enabled
+default_principal =
+default_secret =
+
+# Optional Docker Image to run on slave before running the command
+# This image should be accessible from mesos slave i.e mesos slave
+# should be able to pull this docker image before executing the command.
+docker_image_slave = avhost/docker-airflow

Review comment:
       This default probably needs changing :) Does it make sense to have a default here?

##########
File path: airflow/config_templates/default_test.cfg
##########
@@ -100,6 +100,19 @@ flower_port = 5555
 default_queue = default
 sync_parallelism = 0
 
+[mesos]
+master = localhost:5050
+framework_name = Airflow
+task_cpu = 1
+task_memory = 20
+checkpoint = False
+authenticate = False
+docker_image_slave = test/docker-airflow
+docker_volume_dag_driver = "local"
+docker_volume_dag_name = "airflowdags"
+docker_volume_dag_container_path = "/usr/local/airflow/dags"
+docker_sock = "/var/run/docker.sock"

Review comment:
       Anything that is the default value please don't specify in the test config.

##########
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')

Review comment:
       This isn't needed: as there is a value for this config in the default_airflow.cfg, because of how airflow parses the config (it parsers user file, and layers it over the top of the default file) you can guarantee that this config value will be present.

##########
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('"', '')

Review comment:
       What are all the `replace()` here for?

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

Review comment:
       What happens if you run more than 35535 tasks? This port number will overflow

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

Review comment:
       How does this behave when more than one executor is running?

##########
File path: airflow/configuration.py
##########
@@ -228,6 +228,7 @@ def _validate_config_dependencies(self):
         is_executor_without_sqlite_support = self.get("core", "executor") not in (
             'DebugExecutor',
             'SequentialExecutor',
+            'MesosExecutor',

Review comment:
       This is asying MesosExecutor does support Sqlite. Given the workers are on remote nodes with no access to the .db file, and workers update state in the DB currently, I don't think this is true :)

##########
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',

Review comment:
       Using shell seems like a security risk -- we have to worry about escaping then. Is there another option?

##########
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:
       Why do we need to mount the docker socket in the running slave image?

##########
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'}
+                        ],
+                    },
+                },
+            }
+
+            option = {'Filters': {'RefuseSeconds': '0.5'}}
+
+            tasks.append(task)
+            remaining_cpus -= self.task_cpu
+            remaining_mem -= self.task_mem
+        mesos_offer.accept(tasks, option)
+
+    @provide_session
+    def subscribed(self, driver, session=None):
+        """
+        Subscribe to Mesos Master
+
+        :param driver: Mesos driver object
+        """
+        from airflow.models import Connection
+
+        # Update the Framework ID in the database.
+        conn_id = FRAMEWORK_CONNID_PREFIX + get_framework_name()
+        connection = session.query(Connection).filter_by(conn_id=conn_id).first()
+        if connection is None:
+            connection = Connection(conn_id=conn_id, conn_type='mesos_framework-id', extra=driver.frameworkId)
+        else:
+            connection.extra = driver.frameworkId
+
+        session.commit()
+        self.driver = driver
+
+    def status_update(self, update):
+        """Update the Status of the Tasks. Based by Mesos Events."""
+        task_id = update["status"]["task_id"]["value"]
+        task_state = update["status"]["state"]
+
+        self.log.info("Task %s is in state %s", task_id, task_state)
+
+        try:
+            key = self.task_key_map[task_id]
+
+        except KeyError:
+            # The map may not contain an item if the framework re-registered
+            # after a failover.
+            # Discard these tasks.
+            self.log.info("Unrecognised task key %s", task_id)
+            return
+
+        if task_state == "TASK_FINISHED":
+            self.result_queue.put((key, State.SUCCESS))
+            return
+
+        if task_state in ('TASK_LOST', 'TASK_KILLED', 'TASK_FAILED'):
+            self.result_queue.put((key, State.FAILED))
+            return
+
+
+class MesosExecutor(BaseExecutor):
+    """
+    MesosExecutor allows distributing the execution of task
+    instances to multiple mesos workers.
+
+    Apache Mesos is a distributed systems kernel which abstracts
+    CPU, memory, storage, and other compute resources away from
+    machines (physical or virtual), enabling fault-tolerant and
+    elastic distributed systems to easily be built and run effectively.
+    See http://mesos.apache.org/
+    """
+
+    class MesosFramework(threading.Thread):
+        """MesosFramework class to start the threading"""
+
+        def __init__(self, client):
+            threading.Thread.__init__(self)

Review comment:
       ```suggestion
               super().__init__(self)
   ```

##########
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'}
+                        ],
+                    },
+                },
+            }
+
+            option = {'Filters': {'RefuseSeconds': '0.5'}}
+
+            tasks.append(task)
+            remaining_cpus -= self.task_cpu
+            remaining_mem -= self.task_mem
+        mesos_offer.accept(tasks, option)
+
+    @provide_session
+    def subscribed(self, driver, session=None):
+        """
+        Subscribe to Mesos Master
+
+        :param driver: Mesos driver object
+        """
+        from airflow.models import Connection
+
+        # Update the Framework ID in the database.
+        conn_id = FRAMEWORK_CONNID_PREFIX + get_framework_name()
+        connection = session.query(Connection).filter_by(conn_id=conn_id).first()
+        if connection is None:
+            connection = Connection(conn_id=conn_id, conn_type='mesos_framework-id', extra=driver.frameworkId)
+        else:
+            connection.extra = driver.frameworkId
+
+        session.commit()

Review comment:
       ```suggestion
   ```
   
   Don't use `session.commit()` inside a function with provide_session - https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#database-session-handling

##########
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'}
+                        ],
+                    },
+                },
+            }
+
+            option = {'Filters': {'RefuseSeconds': '0.5'}}
+
+            tasks.append(task)
+            remaining_cpus -= self.task_cpu
+            remaining_mem -= self.task_mem
+        mesos_offer.accept(tasks, option)
+
+    @provide_session
+    def subscribed(self, driver, session=None):
+        """
+        Subscribe to Mesos Master
+
+        :param driver: Mesos driver object
+        """
+        from airflow.models import Connection
+
+        # Update the Framework ID in the database.
+        conn_id = FRAMEWORK_CONNID_PREFIX + get_framework_name()
+        connection = session.query(Connection).filter_by(conn_id=conn_id).first()
+        if connection is None:
+            connection = Connection(conn_id=conn_id, conn_type='mesos_framework-id', extra=driver.frameworkId)
+        else:
+            connection.extra = driver.frameworkId
+
+        session.commit()
+        self.driver = driver
+
+    def status_update(self, update):
+        """Update the Status of the Tasks. Based by Mesos Events."""
+        task_id = update["status"]["task_id"]["value"]
+        task_state = update["status"]["state"]
+
+        self.log.info("Task %s is in state %s", task_id, task_state)
+
+        try:
+            key = self.task_key_map[task_id]
+
+        except KeyError:
+            # The map may not contain an item if the framework re-registered
+            # after a failover.
+            # Discard these tasks.
+            self.log.info("Unrecognised task key %s", task_id)
+            return
+
+        if task_state == "TASK_FINISHED":
+            self.result_queue.put((key, State.SUCCESS))
+            return
+
+        if task_state in ('TASK_LOST', 'TASK_KILLED', 'TASK_FAILED'):
+            self.result_queue.put((key, State.FAILED))
+            return
+
+
+class MesosExecutor(BaseExecutor):
+    """
+    MesosExecutor allows distributing the execution of task
+    instances to multiple mesos workers.
+
+    Apache Mesos is a distributed systems kernel which abstracts
+    CPU, memory, storage, and other compute resources away from
+    machines (physical or virtual), enabling fault-tolerant and
+    elastic distributed systems to easily be built and run effectively.
+    See http://mesos.apache.org/
+    """
+
+    class MesosFramework(threading.Thread):
+        """MesosFramework class to start the threading"""
+
+        def __init__(self, client):
+            threading.Thread.__init__(self)
+            self.client = client
+            self.stop = False
+
+        def run(self):
+            try:
+                self.client.register()
+            except KeyboardInterrupt:
+                print("Stop requested by user, stopping framework....")
+
+    def __init__(self):
+        super().__init__()
+        self.commands_to_run = []
+        self.task_queue = Queue()
+        self.result_queue = Queue()
+        self.driver = None
+        self.client = None
+        self.mesos_framework = None
+
+    @provide_session
+    def start(self, session=None):
+        """Setup and start routine to connect with the mesos master"""
+        if not conf.get('mesos', 'MASTER'):
+            self.log.error("Expecting mesos master URL for mesos executor")
+            raise AirflowException("mesos.master not provided for mesos executor")

Review comment:
       ```suggestion
   ```
   
   see previous comment and config loading




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