├── .gitignore ├── .travis.yml ├── LICENSE ├── README.rst ├── examples ├── README.md ├── executor.py ├── operator_v1.py └── scheduler.py ├── pymesos ├── __init__.py ├── detector.py ├── executor.py ├── interface.py ├── operator_v1.py ├── process.py ├── scheduler.py ├── subprocess │ ├── __init__.py │ ├── executor.py │ ├── popen.py │ └── scheduler.py ├── utils.py └── zkpython.py ├── requirements.txt ├── setup.cfg ├── setup.py ├── tests ├── test_executor.py ├── test_operator.py └── test_scheduler.py └── tox.ini /.gitignore: -------------------------------------------------------------------------------- 1 | *.py[co] 2 | /*.egg-info/ 3 | /*.egg/ 4 | /.cache/ 5 | /.coverage 6 | /.eggs/ 7 | /.idea/ 8 | /.tox/ 9 | /build/ 10 | /dist/ 11 | /venv/ 12 | .vscode/ 13 | Pipfile 14 | Pipfile.lock -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: python 2 | cache: 3 | directories: 4 | $HOME/.cache/pip 5 | sudo: false 6 | addons: 7 | apt: 8 | packages: 9 | - libzookeeper-mt-dev 10 | matrix: 11 | include: 12 | - python: 2.7 13 | env: TOX_ENV=py27 14 | - python: 3.7 15 | env: TOX_ENV=py37 16 | - python: 3.8 17 | env: TOX_ENV=py38 18 | - python: pypy 19 | env: TOX_ENV=pypy 20 | - python: 3.7 21 | env: TOX_ENV=flake8 22 | install: 23 | - pip install -U pip 24 | - pip install tox 25 | script: 26 | - tox -e $TOX_ENV 27 | deploy: 28 | provider: pypi 29 | skip_existing: true 30 | user: windreamer 31 | password: 32 | secure: oOvA3FYYL518XSnZwbGfykxpcqpEIbDJTAh+sg3WeoIXDZzYG6vHkEDBpUDVZb/QEKQ9SlAiPDkZwT3I0b5IN7+pf//UHrN/By99ENqNpK3W9I1GYIOAO6NUwqM3/hmAROppA7Qs5C7A3b++f+0RCLvuf2xQYjQTttU+QI5zc9U= 33 | on: 34 | tags: true 35 | repo: douban/pymesos 36 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | BSD 3-Clause License 2 | 3 | Copyright (c) 2018, Douban Inc. 4 | All rights reserved. 5 | 6 | Redistribution and use in source and binary forms, with or without 7 | modification, are permitted provided that the following conditions are met: 8 | 9 | * Redistributions of source code must retain the above copyright notice, this 10 | list of conditions and the following disclaimer. 11 | 12 | * Redistributions in binary form must reproduce the above copyright notice, 13 | this list of conditions and the following disclaimer in the documentation 14 | and/or other materials provided with the distribution. 15 | 16 | * Neither the name of the copyright holder nor the names of its 17 | contributors may be used to endorse or promote products derived from 18 | this software without specific prior written permission. 19 | 20 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" 21 | AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 22 | IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE 23 | DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE 24 | FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL 25 | DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR 26 | SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 27 | CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, 28 | OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 29 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | -------------------------------------------------------------------------------- /README.rst: -------------------------------------------------------------------------------- 1 | PyMesos 2 | ======== 3 | 4 | .. image:: https://img.shields.io/travis/douban/pymesos.svg 5 | :target: https://travis-ci.org/douban/pymesos 6 | 7 | 8 | .. image:: https://badge.fury.io/py/pymesos.svg 9 | :target: https://badge.fury.io/py/pymesos 10 | 11 | 12 | A pure python implementation of Apache Mesos scheduler, executor and operator. 13 | 14 | Note: 15 | ------ 16 | If you encounter errors when install zkpython, please do ``suto apt install python-dev libzookeeper-st-dev libzookeeper-mt-dev`` first. 17 | 18 | Since ``PyMesos 0.2.0``, ``Apache Mesos`` HTTP API is implemented instead of the ``Protobuf`` API. 19 | 20 | Users of ``PyMesos`` would be able to get rid of the unnecessary dependence of ``Protobuf``. 21 | 22 | Meanwhile, ``PyMesos`` user need to migrate original code to use plain ``Python`` dicts instead of ``Protobuf`` objects to use ``PyMesos >= 0.2.0``. 23 | 24 | For more detail, please refer to `Scheduler HTTP API Document `_ , `Executor HTTP API Document `_ and `Operator HTTP API Document `_ 25 | -------------------------------------------------------------------------------- /examples/README.md: -------------------------------------------------------------------------------- 1 | # Introduction 2 | 3 | This directory contains a minimal Mesos framework using `pymesos` and `addict`. 4 | 5 | 6 | # Acknowledgment 7 | 8 | This framework is based on the example from Chapter 10 of `Mesos in action`. 9 | -------------------------------------------------------------------------------- /examples/executor.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env python2.7 2 | from __future__ import print_function 3 | 4 | import sys 5 | import time 6 | from threading import Thread 7 | 8 | from pymesos import MesosExecutorDriver, Executor, decode_data 9 | from addict import Dict 10 | 11 | 12 | class MinimalExecutor(Executor): 13 | def launchTask(self, driver, task): 14 | def run_task(task): 15 | update = Dict() 16 | update.task_id.value = task.task_id.value 17 | update.state = 'TASK_RUNNING' 18 | update.timestamp = time.time() 19 | driver.sendStatusUpdate(update) 20 | 21 | print(decode_data(task.data), file=sys.stderr) 22 | time.sleep(30) 23 | 24 | update = Dict() 25 | update.task_id.value = task.task_id.value 26 | update.state = 'TASK_FINISHED' 27 | update.timestamp = time.time() 28 | driver.sendStatusUpdate(update) 29 | 30 | thread = Thread(target=run_task, args=(task,)) 31 | thread.start() 32 | 33 | 34 | if __name__ == '__main__': 35 | import logging 36 | logging.basicConfig(level=logging.DEBUG) 37 | driver = MesosExecutorDriver(MinimalExecutor(), use_addict=True) 38 | driver.run() 39 | -------------------------------------------------------------------------------- /examples/operator_v1.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env python2.7 2 | from __future__ import print_function 3 | 4 | import sys 5 | 6 | from pymesos import MesosOperatorMasterDriver, OperatorMaster 7 | 8 | 9 | class MinimalOperator(OperatorMaster): 10 | def __init__(self): 11 | pass 12 | 13 | def taskAdded(self, task_info): 14 | logging.debug('Task added') 15 | logging.debug(task_info) 16 | 17 | def taskUpdated(self, task_info): 18 | logging.debug('Task updated') 19 | logging.debug(task_info) 20 | 21 | def frameworkAdded(self, framework_info): 22 | logging.debug('Framework added') 23 | logging.debug(framework_info) 24 | 25 | def frameworkUpdated(self, framework_info): 26 | logging.debug('Framework updated') 27 | logging.debug(framework_info) 28 | 29 | def frameworkRemoved(self, framework_info): 30 | logging.debug('Framework removed') 31 | logging.debug(framework_info) 32 | 33 | def agentAdded(self, agent_info): 34 | logging.debug('Agent added') 35 | logging.debug(agent_info) 36 | 37 | def agentRemoved(self, agent_info): 38 | logging.debug('Agent removed') 39 | logging.debug(agent_info) 40 | 41 | 42 | def main(master): 43 | driver = MesosOperatorMasterDriver(master, MinimalOperator()) 44 | res = driver.getHealth() 45 | print(res) 46 | driver.run() 47 | 48 | 49 | if __name__ == '__main__': 50 | import logging 51 | logging.basicConfig(level=logging.DEBUG) 52 | if len(sys.argv) != 2: 53 | print("Usage: {} ".format(sys.argv[0])) 54 | sys.exit(1) 55 | else: 56 | main(sys.argv[1]) 57 | -------------------------------------------------------------------------------- /examples/scheduler.py: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env python2.7 2 | from __future__ import print_function 3 | 4 | import sys 5 | import uuid 6 | import time 7 | import socket 8 | import signal 9 | import getpass 10 | from threading import Thread 11 | from os.path import abspath, join, dirname 12 | 13 | from pymesos import MesosSchedulerDriver, Scheduler, encode_data 14 | from addict import Dict 15 | 16 | TASK_CPU = 0.1 17 | TASK_MEM = 32 18 | EXECUTOR_CPUS = 0.1 19 | EXECUTOR_MEM = 32 20 | 21 | 22 | class MinimalScheduler(Scheduler): 23 | 24 | def __init__(self, executor): 25 | self.executor = executor 26 | 27 | def resourceOffers(self, driver, offers): 28 | filters = {'refuse_seconds': 5} 29 | 30 | for offer in offers: 31 | cpus = self.getResource(offer.resources, 'cpus') 32 | mem = self.getResource(offer.resources, 'mem') 33 | if cpus < TASK_CPU or mem < TASK_MEM: 34 | continue 35 | 36 | task = Dict() 37 | task_id = str(uuid.uuid4()) 38 | task.task_id.value = task_id 39 | task.agent_id.value = offer.agent_id.value 40 | task.name = 'task {}'.format(task_id) 41 | task.executor = self.executor 42 | task.data = encode_data('Hello from task {}!'.format(task_id)) 43 | 44 | task.resources = [ 45 | dict(name='cpus', type='SCALAR', scalar={'value': TASK_CPU}), 46 | dict(name='mem', type='SCALAR', scalar={'value': TASK_MEM}), 47 | ] 48 | 49 | driver.launchTasks(offer.id, [task], filters) 50 | 51 | def getResource(self, res, name): 52 | for r in res: 53 | if r.name == name: 54 | return r.scalar.value 55 | return 0.0 56 | 57 | def statusUpdate(self, driver, update): 58 | logging.debug('Status update TID %s %s', 59 | update.task_id.value, 60 | update.state) 61 | 62 | 63 | def main(master): 64 | executor = Dict() 65 | executor.executor_id.value = 'MinimalExecutor' 66 | executor.name = executor.executor_id.value 67 | executor.command.value = '%s %s' % ( 68 | sys.executable, 69 | abspath(join(dirname(__file__), 'executor.py')) 70 | ) 71 | executor.resources = [ 72 | dict(name='mem', type='SCALAR', scalar={'value': EXECUTOR_MEM}), 73 | dict(name='cpus', type='SCALAR', scalar={'value': EXECUTOR_CPUS}), 74 | ] 75 | 76 | framework = Dict() 77 | framework.user = getpass.getuser() 78 | framework.name = "MinimalFramework" 79 | framework.hostname = socket.gethostname() 80 | 81 | driver = MesosSchedulerDriver( 82 | MinimalScheduler(executor), 83 | framework, 84 | master, 85 | use_addict=True, 86 | ) 87 | 88 | def signal_handler(signal, frame): 89 | driver.stop() 90 | 91 | def run_driver_thread(): 92 | driver.run() 93 | 94 | driver_thread = Thread(target=run_driver_thread, args=()) 95 | driver_thread.start() 96 | 97 | print('Scheduler running, Ctrl+C to quit.') 98 | signal.signal(signal.SIGINT, signal_handler) 99 | 100 | while driver_thread.is_alive(): 101 | time.sleep(1) 102 | 103 | 104 | if __name__ == '__main__': 105 | import logging 106 | logging.basicConfig(level=logging.DEBUG) 107 | if len(sys.argv) != 2: 108 | print("Usage: {} ".format(sys.argv[0])) 109 | sys.exit(1) 110 | else: 111 | main(sys.argv[1]) 112 | -------------------------------------------------------------------------------- /pymesos/__init__.py: -------------------------------------------------------------------------------- 1 | import sys 2 | import logging 3 | 4 | from .interface import Scheduler, Executor, OperatorMaster 5 | from .scheduler import MesosSchedulerDriver 6 | from .executor import MesosExecutorDriver 7 | from .operator_v1 import MesosOperatorMasterDriver, MesosOperatorAgentDriver 8 | from .utils import encode_data, decode_data 9 | 10 | 11 | LOG_FMT = '%(asctime)-15s [%(levelname)s] [%(threadName)s]' \ 12 | ' [%(name)-9s:%(lineno)d] %(message)s' 13 | DATE_FMT = '%Y-%m-%d %H:%M:%S' 14 | 15 | logger = logging.getLogger('pymesos') 16 | handler = logging.StreamHandler(stream=sys.stderr) 17 | handler.setFormatter(logging.Formatter(LOG_FMT, DATE_FMT)) 18 | logger.addHandler(handler) 19 | logger.setLevel(logging.INFO) 20 | 21 | 22 | __VERSION__ = '0.3.15' 23 | 24 | __all__ = ( 25 | 'Scheduler', 26 | 'MesosSchedulerDriver', 27 | 'Executor', 28 | 'MesosExecutorDriver', 29 | 'encode_data', 30 | 'decode_data', 31 | 'OperatorMaster', 32 | 'MesosOperatorMasterDriver', 33 | 'MesosOperatorAgentDriver', 34 | ) 35 | -------------------------------------------------------------------------------- /pymesos/detector.py: -------------------------------------------------------------------------------- 1 | from __future__ import print_function 2 | 3 | try: 4 | from zookeeper import ZooKeeperException as ZookeeperError 5 | from .zkpython import ZKClient, ChildrenWatch, DataWatch 6 | 7 | def adjust_zk_logging_level(): 8 | pass 9 | except ImportError: 10 | from kazoo.client import KazooClient as ZKClient 11 | from kazoo.recipe.watchers import ChildrenWatch, DataWatch 12 | from kazoo.exceptions import ZookeeperError 13 | 14 | def adjust_zk_logging_level(): 15 | import logging 16 | import kazoo 17 | kazoo.client.log.setLevel(logging.WARNING) 18 | kazoo.protocol.connection.log.setLevel(logging.WARNING) 19 | 20 | 21 | class MasterDetector(object): 22 | 23 | def __init__(self, uri, agent): 24 | self.uri = uri 25 | self.agent = agent 26 | self.zk = ZKClient(uri, 10) 27 | self.masterSeq = None 28 | 29 | def choose(self, children): 30 | children = [child for child in children if child != 'log_replicas'] 31 | if not children: 32 | self.agent.onNoMasterDetectedMessage() 33 | return True 34 | masterSeq = min(children) 35 | if masterSeq == self.masterSeq: 36 | return True 37 | self.masterSeq = masterSeq 38 | DataWatch(self.zk, '/' + masterSeq, self.notify) 39 | return True 40 | 41 | def notify(self, master_addr, _): 42 | self.agent.onNewMasterDetectedMessage(master_addr) 43 | return False 44 | 45 | def start(self): 46 | adjust_zk_logging_level() 47 | self.zk.start() 48 | try: 49 | ChildrenWatch(self.zk, '', self.choose) 50 | except ZookeeperError: 51 | self.agent.onNoMasterDetectedMessage() 52 | self.stop() 53 | 54 | def stop(self): 55 | try: 56 | self.zk.stop() 57 | except Exception: 58 | pass 59 | 60 | 61 | def test(): 62 | import time 63 | import logging 64 | logging.basicConfig() 65 | 66 | class Agent: 67 | 68 | def onNewMasterDetectedMessage(self, addr): 69 | print(('got', addr)) 70 | 71 | def onNoMasterDetectedMessage(self): 72 | print('no master') 73 | d = MasterDetector('zk1:2181/mesos_master2', Agent()) 74 | d.start() 75 | time.sleep(60) 76 | d.stop() 77 | 78 | 79 | if __name__ == '__main__': 80 | test() 81 | -------------------------------------------------------------------------------- /pymesos/executor.py: -------------------------------------------------------------------------------- 1 | import os 2 | import json 3 | import time 4 | import uuid 5 | import signal 6 | import logging 7 | from threading import Thread 8 | from addict import Dict 9 | from six.moves.http_client import HTTPConnection 10 | from .process import Process 11 | from .interface import ExecutorDriver 12 | from .utils import DAY, parse_duration, encode_data, decode_data 13 | 14 | logger = logging.getLogger(__name__) 15 | 16 | 17 | class MesosExecutorDriver(Process, ExecutorDriver): 18 | def __init__(self, executor, use_addict=False, timeout=DAY, 19 | http_timeout=10): 20 | env = os.environ 21 | agent_endpoint = env['MESOS_AGENT_ENDPOINT'] 22 | super(MesosExecutorDriver, self).__init__(master=agent_endpoint, 23 | timeout=timeout) 24 | 25 | framework_id = env['MESOS_FRAMEWORK_ID'] 26 | assert framework_id 27 | self.framework_id = dict(value=framework_id) 28 | executor_id = env['MESOS_EXECUTOR_ID'] 29 | self.executor_id = dict(value=executor_id) 30 | grace_shutdown_period = env.get('MESOS_EXECUTOR_SHUTDOWN_GRACE_PERIOD') 31 | if grace_shutdown_period: 32 | self.grace_shutdown_period = parse_duration(grace_shutdown_period) 33 | else: 34 | self.grace_shutdown_period = 0.0 35 | 36 | self.checkpoint = bool(env.get('MESOS_CHECKPOINT')) 37 | self.local = bool(env.get('MESOS_LOCAL')) 38 | 39 | self.executor = executor 40 | self.framework_info = None 41 | self.executor_info = None 42 | self.tasks = {} 43 | self.updates = {} 44 | self._conn = None 45 | self._dict_cls = Dict if use_addict else dict 46 | self._http_timeout = http_timeout 47 | 48 | def _delay_kill(self): 49 | def _(): 50 | try: 51 | time.sleep(self.grace_shutdown_period) 52 | os.killpg(0, signal.SIGKILL) 53 | except Exception: 54 | logger.exception('Failed to force kill executor') 55 | 56 | t = Thread(target=_) 57 | t.daemon = True 58 | t.start() 59 | 60 | def gen_request(self): 61 | body = json.dumps(dict( 62 | type='SUBSCRIBE', 63 | framework_id=self.framework_id, 64 | executor_id=self.executor_id, 65 | subscribe=dict( 66 | unacknowledged_tasks=list(self.tasks.values()), 67 | unacknowledged_updates=list(self.updates.values()), 68 | ), 69 | )) 70 | 71 | request = ('POST /api/v1/executor HTTP/1.1\r\nHost: %s\r\n' 72 | 'Content-Type: application/json\r\n' 73 | 'Accept: application/json\r\n' 74 | 'Connection: close\r\nContent-Length: %s\r\n\r\n%s') % ( 75 | self.master, len(body), body 76 | ) 77 | return request.encode('utf-8') 78 | 79 | def start(self): 80 | super(MesosExecutorDriver, self).start() 81 | self._notify() 82 | 83 | def on_close(self): 84 | if self._conn is not None: 85 | self._conn.close() 86 | self._conn = None 87 | self.version = None 88 | 89 | self.executor.disconnected(self) 90 | if not self.checkpoint: 91 | if not self.local: 92 | self._delay_kill() 93 | self.executor.shutdown(self) 94 | self.abort() 95 | 96 | def on_event(self, event): 97 | if 'type' in event: 98 | _type = event['type'].lower() 99 | if _type == 'shutdown': 100 | self.on_shutdown() 101 | return 102 | 103 | if _type == 'heartbeat': 104 | return 105 | 106 | if _type not in event: 107 | logger.error( 108 | 'Missing `%s` in event %s' % 109 | (_type, event)) 110 | return 111 | 112 | event = event[_type] 113 | func_name = 'on_%s' % (_type,) 114 | func = getattr(self, func_name, None) 115 | if func is not None: 116 | func(event) 117 | else: 118 | logger.error('Unknown type:%s, event:%s' % (_type, event)) 119 | else: 120 | logger.error('Unknown event:%s' % (event,)) 121 | 122 | def on_subscribed(self, info): 123 | executor_info = info['executor_info'] 124 | framework_info = info['framework_info'] 125 | agent_info = info['agent_info'] 126 | assert executor_info['executor_id'] == self.executor_id 127 | assert framework_info['id'] == self.framework_id 128 | 129 | if self.executor_info is None or self.framework_info is None: 130 | self.executor_info = executor_info 131 | self.framework_info = framework_info 132 | self.executor.registered( 133 | self, self._dict_cls(executor_info), 134 | self._dict_cls(framework_info), self._dict_cls(agent_info) 135 | ) 136 | else: 137 | self.executor.reregistered(self, self._dict_cls(agent_info)) 138 | 139 | def on_launch(self, event): 140 | task_info = event['task'] 141 | task_id = task_info['task_id']['value'] 142 | assert task_id not in self.tasks 143 | self.tasks[task_id] = task_info 144 | self.executor.launchTask(self, self._dict_cls(task_info)) 145 | 146 | def on_launch_group(self, event): 147 | task_group = event['task_group'] 148 | task_infos = [] 149 | for task_info in task_group['tasks']: 150 | task_id = task_info['task_id']['value'] 151 | assert task_id not in self.tasks 152 | self.tasks[task_id] = task_info 153 | task_infos.append(self._dict_cls(task_info)) 154 | 155 | self.executor.launchTaskGroup(self, task_infos) 156 | 157 | def on_kill(self, event): 158 | task_id = event['task_id'] 159 | self.executor.killTask(self, self._dict_cls(task_id)) 160 | 161 | def on_acknowledged(self, event): 162 | task_id = event['task_id']['value'] 163 | uuid_ = uuid.UUID(bytes=decode_data(event['uuid'])) 164 | self.updates.pop(uuid_, None) 165 | self.tasks.pop(task_id, None) 166 | 167 | def on_message(self, event): 168 | data = event['data'] 169 | self.executor.frameworkMessage(self, data) 170 | 171 | def on_error(self, event): 172 | message = event['message'] 173 | self.executor.error(self, message) 174 | 175 | def on_shutdown(self): 176 | if not self.local: 177 | self._delay_kill() 178 | self.executor.shutdown(self) 179 | self.abort() 180 | 181 | def _get_conn(self): 182 | if not self.connected: 183 | return None 184 | 185 | if self._conn is not None: 186 | return self._conn 187 | 188 | host, port = self.master.split(':', 2) 189 | port = int(port) 190 | self._conn = HTTPConnection(host, port, timeout=self._http_timeout) 191 | return self._conn 192 | 193 | def _send(self, body, path='/api/v1/executor', method='POST', headers={}): 194 | with self._lock: 195 | conn = self._get_conn() 196 | if conn is None: 197 | raise RuntimeError('Not connected yet') 198 | 199 | if body != '': 200 | data = json.dumps(body).encode('utf-8') 201 | headers['Content-Type'] = 'application/json' 202 | else: 203 | data = '' 204 | 205 | stream_id = self.stream_id 206 | if stream_id: 207 | headers['Mesos-Stream-Id'] = stream_id 208 | 209 | try: 210 | conn.request(method, path, body=data, headers=headers) 211 | resp = conn.getresponse() 212 | except Exception: 213 | self._conn.close() 214 | self._conn = None 215 | raise 216 | 217 | if resp.status < 200 or resp.status >= 300: 218 | raise RuntimeError( 219 | 'Failed to send request code=%s, message=%s' % ( 220 | resp.status, resp.read() 221 | ) 222 | ) 223 | 224 | result = resp.read() 225 | if not result: 226 | return {} 227 | 228 | try: 229 | return json.loads(result.decode('utf-8')) 230 | except Exception: 231 | return {} 232 | 233 | def sendStatusUpdate(self, status): 234 | if 'timestamp' not in status: 235 | status['timestamp'] = int(time.time()) 236 | 237 | if 'uuid' not in status: 238 | status['uuid'] = encode_data(uuid.uuid4().bytes) 239 | 240 | if 'source' not in status: 241 | status['source'] = 'SOURCE_EXECUTOR' 242 | 243 | body = dict( 244 | type='UPDATE', 245 | executor_id=self.executor_id, 246 | framework_id=self.framework_id, 247 | update=dict( 248 | status=status, 249 | ), 250 | ) 251 | self._send(body) 252 | 253 | def sendFrameworkMessage(self, data): 254 | body = dict( 255 | type='MESSAGE', 256 | executor_id=self.executor_id, 257 | framework_id=self.framework_id, 258 | message=dict( 259 | data=data, 260 | ), 261 | ) 262 | self._send(body) 263 | -------------------------------------------------------------------------------- /pymesos/interface.py: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | 17 | # See include/mesos/scheduler.hpp, include/mesos/executor.hpp and 18 | # include/mesos/mesos.proto for more information documenting this 19 | # interface. 20 | 21 | """Python bindings for Mesos.""" 22 | 23 | from __future__ import print_function 24 | 25 | import sys 26 | 27 | __all__ = ( 28 | 'Executor', 29 | 'ExecutorDriver', 30 | 'Scheduler', 31 | 'SchedulerDriver', 32 | 'OperatorMasterDriver', 33 | 'OperatorMaster', 34 | 'OperatorAgentDriver', 35 | ) 36 | 37 | class Scheduler(object): 38 | """ 39 | Base class for Mesos schedulers. Users' schedulers should extend this 40 | class to get default implementations of methods they don't override. 41 | """ 42 | 43 | def registered(self, driver, frameworkId, masterInfo): 44 | """ 45 | Invoked when the scheduler successfully registers with a Mesos master. 46 | It is called with the frameworkId, a unique ID generated by the 47 | master, and the masterInfo which is information about the master 48 | itself. 49 | """ 50 | 51 | def reregistered(self, driver, masterInfo): 52 | """ 53 | Invoked when the scheduler re-registers with a newly elected Mesos 54 | master. This is only called when the scheduler has previously been 55 | registered. masterInfo contains information about the newly elected 56 | master. 57 | """ 58 | 59 | def disconnected(self, driver): 60 | """ 61 | Invoked when the scheduler becomes disconnected from the master, e.g. 62 | the master fails and another is taking over. 63 | """ 64 | 65 | def processHeartBeat(self, driver): 66 | """ 67 | Invoked when the scheduler gets a heartbeat 68 | """ 69 | 70 | def resourceOffers(self, driver, offers): 71 | """ 72 | Invoked when resources have been offered to this framework. A single 73 | offer will only contain resources from a single slave. Resources 74 | associated with an offer will not be re-offered to _this_ framework 75 | until either (a) this framework has rejected those resources (see 76 | SchedulerDriver.launchTasks) or (b) those resources have been 77 | rescinded (see Scheduler.offerRescinded). Note that resources may be 78 | concurrently offered to more than one framework at a time (depending 79 | on the allocator being used). In that case, the first framework to 80 | launch tasks using those resources will be able to use them while the 81 | other frameworks will have those resources rescinded (or if a 82 | framework has already launched tasks with those resources then those 83 | tasks will fail with a TASK_LOST status and a message saying as much). 84 | """ 85 | 86 | def inverseOffers(self, driver, offers): 87 | """ 88 | Invoked when an inverse offer is sent to the framework. An inverse 89 | offer and a resource offer can hold many of the same fields, but an 90 | inverse offer requests resources rather than offering them. Inverse 91 | offers may be accepted, rejected, re-offered, and rescinded. 92 | """ 93 | 94 | def offerRescinded(self, driver, offerId): 95 | """ 96 | Invoked when an offer is no longer valid (e.g., the slave was lost or 97 | another framework used resources in the offer.) If for whatever reason 98 | an offer is never rescinded (e.g., dropped message, failing over 99 | framework, etc.), a framework that attempts to launch tasks using an 100 | invalid offer will receive TASK_LOST status updates for those tasks 101 | (see Scheduler.resourceOffers). 102 | """ 103 | 104 | def inverseOfferRescinded(self, driver, offerId): 105 | """ 106 | Invoked when an inverse offer is no longer valid (e.g., the slave was lost or 107 | another framework used resources in the inverse offer.) If for whatever reason 108 | an inverse offer is never rescinded (e.g., dropped message, failing over 109 | framework, etc.), a framework that attempts to launch tasks using an 110 | invalid inverse offer will receive TASK_LOST status updates for those tasks 111 | (see Scheduler.resourceOffers). 112 | """ 113 | 114 | def statusUpdate(self, driver, status): 115 | """ 116 | Invoked when the status of a task has changed (e.g., a slave is 117 | lost and so the task is lost, a task finishes and an executor 118 | sends a status update saying so, etc). If implicit 119 | acknowledgements are being used, then returning from this 120 | callback _acknowledges_ receipt of this status update! If for 121 | whatever reason the scheduler aborts during this callback (or 122 | the process exits) another status update will be delivered (note, 123 | however, that this is currently not true if the slave sending the 124 | status update is lost/fails during that time). If explicit 125 | acknowledgements are in use, the scheduler must acknowledge this 126 | status on the driver. 127 | """ 128 | 129 | def operationStatusUpdate(self, driver, status): 130 | """ 131 | Invoked when there is an operation status update generated by the 132 | master, agent, or resource provider. These updates are only sent to 133 | the framework for operations which had the operation ID set by the 134 | framework. It is the responsibility of the scheduler to explicitly 135 | acknowledge the receipt of a status update. 136 | """ 137 | 138 | def frameworkMessage(self, driver, executorId, slaveId, message): 139 | """ 140 | Invoked when an executor sends a message. These messages are best 141 | effort; do not expect a framework message to be retransmitted in any 142 | reliable fashion. 143 | """ 144 | 145 | def slaveLost(self, driver, slaveId): 146 | """ 147 | Invoked when a slave has been determined unreachable (e.g., machine 148 | failure, network partition.) Most frameworks will need to reschedule 149 | any tasks launched on this slave on a new slave. 150 | 151 | NOTE: This callback is not reliably delivered. If a host or 152 | network failure causes messages between the master and the 153 | scheduler to be dropped, this callback may not be invoked. 154 | """ 155 | 156 | def executorLost(self, driver, executorId, slaveId, status): 157 | """ 158 | Invoked when an executor has exited/terminated. Note that any tasks 159 | running will have TASK_LOST status updates automatically generated. 160 | 161 | NOTE: This callback is not reliably delivered. If a host or 162 | network failure causes messages between the master and the 163 | scheduler to be dropped, this callback may not be invoked. 164 | """ 165 | 166 | def error(self, driver, message): 167 | """ 168 | Invoked when there is an unrecoverable error in the scheduler or 169 | scheduler driver. The driver will be aborted BEFORE invoking this 170 | callback. 171 | """ 172 | print("Error from Mesos: %s " % message, file=sys.stderr) 173 | 174 | 175 | class SchedulerDriver(object): 176 | """ 177 | Interface for Mesos scheduler drivers. Users may wish to implement this 178 | class in mock objects for tests. 179 | """ 180 | def start(self): 181 | """ 182 | Starts the scheduler driver. This needs to be called before any other 183 | driver calls are made. 184 | """ 185 | 186 | def stop(self, failover=False): 187 | """ 188 | Stops the scheduler driver. If the 'failover' flag is set to False 189 | then it is expected that this framework will never reconnect to Mesos 190 | and all of its executors and tasks can be terminated. Otherwise, all 191 | executors and tasks will remain running (for some framework specific 192 | failover timeout) allowing the scheduler to reconnect (possibly in the 193 | same process, or from a different process, for example, on a different 194 | machine.) 195 | """ 196 | 197 | def abort(self): 198 | """ 199 | Aborts the driver so that no more callbacks can be made to the 200 | scheduler. The semantics of abort and stop have deliberately been 201 | separated so that code can detect an aborted driver (i.e., via the 202 | return status of SchedulerDriver.join), and instantiate and start 203 | another driver if desired (from within the same process.) 204 | """ 205 | 206 | def join(self): 207 | """ 208 | Waits for the driver to be stopped or aborted, possibly blocking the 209 | current thread indefinitely. The return status of this function can 210 | be used to determine if the driver was aborted (see mesos.proto for a 211 | description of Status). 212 | """ 213 | 214 | def run(self): 215 | """ 216 | Starts and immediately joins (i.e., blocks on) the driver. 217 | """ 218 | 219 | def requestResources(self, requests): 220 | """ 221 | Requests resources from Mesos (see mesos.proto for a description of 222 | Request and how, for example, to request resources from specific 223 | slaves.) Any resources available are offered to the framework via 224 | Scheduler.resourceOffers callback, asynchronously. 225 | """ 226 | 227 | def launchTasks(self, offerIds, tasks, filters=None): 228 | """ 229 | Launches the given set of tasks. Any remaining resources (i.e., 230 | those that are not used by the launched tasks or their executors) 231 | will be considered declined. Note that this includes resources 232 | used by tasks that the framework attempted to launch but failed 233 | (with TASK_ERROR) due to a malformed task description. The 234 | specified filters are applied on all unused resources (see 235 | mesos.proto for a description of Filters). Available resources 236 | are aggregated when multiple offers are provided. Note that all 237 | offers must belong to the same slave. Invoking this function with 238 | an empty collection of tasks declines offers in their entirety 239 | (see Scheduler.declineOffer). Note that passing a single offer 240 | is also supported. 241 | """ 242 | 243 | def killTask(self, taskId): 244 | """ 245 | Kills the specified task. Note that attempting to kill a task is 246 | currently not reliable. If, for example, a scheduler fails over while 247 | it was attempting to kill a task it will need to retry in the future. 248 | Likewise, if unregistered / disconnected, the request will be dropped 249 | dropped (these semantics may be changed in the future). 250 | """ 251 | 252 | def acceptOffers(self, offerIds, operations, filters=None): 253 | """ 254 | Accepts the given offers and performs a sequence of operations on 255 | those accepted offers. See Offer.Operation in mesos.proto for the 256 | set of available operations. Any remaining resources (i.e., those 257 | that are not used by the launched tasks or their executors) will 258 | be considered declined. Note that this includes resources used by 259 | tasks that the framework attempted to launch but failed (with 260 | TASK_ERROR) due to a malformed task description. The specified 261 | filters are applied on all unused resources (see mesos.proto for 262 | a description of Filters). Available resources are aggregated 263 | when multiple offers are provided. Note that all offers must 264 | belong to the same slave. 265 | """ 266 | 267 | def acceptInverseOffers(self, offer_ids, filters=None): 268 | """ 269 | Accepts an inverse offer. Inverse offers should only be accepted 270 | if the resources in the offer can be safely evacuated before the 271 | provided unavailability. 272 | """ 273 | 274 | def declineOffer(self, offerId, filters=None): 275 | """ 276 | Declines an offer in its entirety and applies the specified 277 | filters on the resources (see mesos.proto for a description of 278 | Filters). Note that this can be done at any time, it is not 279 | necessary to do this within the Scheduler::resourceOffers 280 | callback. 281 | """ 282 | 283 | def declineInverseOffer(self, offer_ids, filters=None): 284 | """ 285 | Declines an inverse offer. Inverse offers should be declined if 286 | the resources in the offer might not be safely evacuated before 287 | the provided unavailability. 288 | """ 289 | 290 | def reviveOffers(self): 291 | """ 292 | Removes all filters previously set by the framework (via 293 | launchTasks()). This enables the framework to receive offers from 294 | those filtered slaves. 295 | """ 296 | 297 | def suppressOffers(self): 298 | """ 299 | Inform Mesos master to stop sending offers to the framework. The 300 | scheduler should call reviveOffers() to resume getting offers. 301 | """ 302 | 303 | def acknowledgeStatusUpdate(self, status): 304 | """ 305 | Acknowledges the status update. This should only be called 306 | once the status update is processed durably by the scheduler. 307 | Not that explicit acknowledgements must be requested via the 308 | constructor argument, otherwise a call to this method will 309 | cause the driver to crash. 310 | """ 311 | 312 | def acknowledgeOperationStatusUpdate(self, status): 313 | """ 314 | Acknowledges the operation status update. This should only be called 315 | once the operation status update is processed durably by the scheduler. 316 | Not that explicit acknowledgements must be requested via the 317 | constructor argument, otherwise a call to this method will 318 | cause the driver to crash. 319 | """ 320 | 321 | def sendFrameworkMessage(self, executorId, slaveId, data): 322 | """ 323 | Sends a message from the framework to one of its executors. These 324 | messages are best effort; do not expect a framework message to be 325 | retransmitted in any reliable fashion. 326 | """ 327 | 328 | def reconcileTasks(self, tasks): 329 | """ 330 | Allows the framework to query the status for non-terminal tasks. 331 | This causes the master to send back the latest task status for 332 | each task in 'statuses', if possible. Tasks that are no longer 333 | known will result in a TASK_LOST update. If statuses is empty, 334 | then the master will send the latest status for each task 335 | currently known. 336 | """ 337 | 338 | class Executor(object): 339 | """ 340 | Base class for Mesos executors. Users' executors should extend this 341 | class to get default implementations of methods they don't override. 342 | """ 343 | 344 | def registered(self, driver, executorInfo, frameworkInfo, slaveInfo): 345 | """ 346 | Invoked once the executor driver has been able to successfully connect 347 | with Mesos. In particular, a scheduler can pass some data to its 348 | executors through the FrameworkInfo.ExecutorInfo's data field. 349 | """ 350 | 351 | def reregistered(self, driver, slaveInfo): 352 | """ 353 | Invoked when the executor re-registers with a restarted slave. 354 | """ 355 | 356 | def disconnected(self, driver): 357 | """ 358 | Invoked when the executor becomes "disconnected" from the slave (e.g., 359 | the slave is being restarted due to an upgrade). 360 | """ 361 | 362 | def launchTask(self, driver, task): 363 | """ 364 | Invoked when a task has been launched on this executor (initiated via 365 | Scheduler.launchTasks). Note that this task can be realized with a 366 | thread, a process, or some simple computation, however, no other 367 | callbacks will be invoked on this executor until this callback has 368 | returned. 369 | """ 370 | 371 | def launchTaskGroup(self, driver, task_infos): 372 | """ 373 | Invoked when a task group has been launched on this executor (initiated via 374 | Scheduler.launchTasks). Note that this task can be realized with a 375 | thread, a process, or some simple computation, however, no other 376 | callbacks will be invoked on this executor until this callback has 377 | returned. 378 | """ 379 | 380 | def killTask(self, driver, taskId): 381 | """ 382 | Invoked when a task running within this executor has been killed (via 383 | SchedulerDriver.killTask). Note that no status update will be sent on 384 | behalf of the executor, the executor is responsible for creating a new 385 | TaskStatus (i.e., with TASK_KILLED) and invoking ExecutorDriver's 386 | sendStatusUpdate. 387 | """ 388 | 389 | def frameworkMessage(self, driver, message): 390 | """ 391 | Invoked when a framework message has arrived for this executor. These 392 | messages are best effort; do not expect a framework message to be 393 | retransmitted in any reliable fashion. 394 | """ 395 | 396 | def shutdown(self, driver): 397 | """ 398 | Invoked when the executor should terminate all of its currently 399 | running tasks. Note that after Mesos has determined that an executor 400 | has terminated any tasks that the executor did not send terminal 401 | status updates for (e.g., TASK_KILLED, TASK_FINISHED, TASK_FAILED, 402 | etc) a TASK_LOST status update will be created. 403 | """ 404 | 405 | def error(self, driver, message): 406 | """ 407 | Invoked when a fatal error has occurred with the executor and/or 408 | executor driver. The driver will be aborted BEFORE invoking this 409 | callback. 410 | """ 411 | print("Error from Mesos: %s" % message, file=sys.stderr) 412 | 413 | 414 | 415 | class ExecutorDriver(object): 416 | """ 417 | Interface for Mesos executor drivers. Users may wish to extend this 418 | class in mock objects for tests. 419 | """ 420 | def start(self): 421 | """ 422 | Starts the executor driver. This needs to be called before any other 423 | driver calls are made. 424 | """ 425 | 426 | def stop(self): 427 | """ 428 | Stops the executor driver. 429 | """ 430 | 431 | def abort(self): 432 | """ 433 | Aborts the driver so that no more callbacks can be made to the 434 | executor. The semantics of abort and stop have deliberately been 435 | separated so that code can detect an aborted driver (i.e., via the 436 | return status of ExecutorDriver.join), and instantiate and start 437 | another driver if desired (from within the same process, although this 438 | functionality is currently not supported for executors). 439 | """ 440 | 441 | def join(self): 442 | """ 443 | Waits for the driver to be stopped or aborted, possibly blocking the 444 | current thread indefinitely. The return status of this function can 445 | be used to determine if the driver was aborted (see mesos.proto for a 446 | description of Status). 447 | """ 448 | 449 | def run(self): 450 | """ 451 | Starts and immediately joins (i.e., blocks on) the driver. 452 | """ 453 | 454 | def sendStatusUpdate(self, status): 455 | """ 456 | Sends a status update to the framework scheduler, retrying as 457 | necessary until an acknowledgement has been received or the executor 458 | is terminated (in which case, a TASK_LOST status update will be sent). 459 | See Scheduler.statusUpdate for more information about status update 460 | acknowledgements. 461 | """ 462 | 463 | def sendFrameworkMessage(self, data): 464 | """ 465 | Sends a message to the framework scheduler. These messages are best 466 | effort; do not expect a framework message to be retransmitted in any 467 | reliable fashion. 468 | """ 469 | 470 | 471 | class OperatorDaemonDriver(object): 472 | """ 473 | Operator HTTP API: Operations common to master daemon and agent daemon. 474 | """ 475 | 476 | def getHealth(self): 477 | """ 478 | This call retrieves the health status of master daemon or agent daemon. 479 | """ 480 | 481 | def getFlags(self): 482 | """ 483 | This call retrieves the daemon's overall flag configuration. 484 | """ 485 | 486 | def getVersion(self): 487 | """ 488 | This call retrieves the daemon's version information. 489 | """ 490 | 491 | def getMetrics(self, timeout): 492 | """ 493 | This call gives the snapshot of current metrics to the end user. If timeout is set in the call, it would be used 494 | to determine the maximum amount of time the API will take to respond. If the timeout is exceeded, some metrics may 495 | not be included in the response. 496 | """ 497 | 498 | def getLoggingLevel(self): 499 | """ 500 | This call retrieves the daemon's logging level. 501 | """ 502 | 503 | def setLoggingLevel(self, level, duration): 504 | """ 505 | Sets the logging verbosity level for a specified duration for master daemon or agent daemon. Mesos uses glog for 506 | logging. The library only uses verbose logging which means nothing will be output unless the verbosity level is 507 | set (by default it's 0, libprocess uses levels 1, 2, and 3). 508 | """ 509 | 510 | def listFiles(self, path): 511 | """ 512 | This call retrieves the file listing for a directory in master daemon or agent daemon. 513 | """ 514 | 515 | def readFile(self, path, length, offset): 516 | """ 517 | Reads data from a file. This call takes path of the file to be read in the daemon, offset to start reading 518 | position and length for the maximum number of bytes to read. 519 | """ 520 | 521 | def getState(self): 522 | """ 523 | This call retrieves the overall cluster state. 524 | """ 525 | 526 | def getFrameworks(self): 527 | """ 528 | This call retrieves information about all the frameworks known to the master daemon or agent daemon. 529 | """ 530 | 531 | def getExecutors(self): 532 | """ 533 | Queries about all the executors known to the master daemon or agent daemon. 534 | """ 535 | 536 | def getTasks(self): 537 | """ 538 | Query about all the tasks known to the master daemon or agent daemon. 539 | """ 540 | 541 | 542 | class OperatorMasterDriver(OperatorDaemonDriver): 543 | """ 544 | Interface for Mesos operator drivers. Users may wish to extend this 545 | class in mock objects for tests. 546 | """ 547 | 548 | def start(self): 549 | """ 550 | Starts the operator driver. This needs to be called before any other 551 | driver calls are made. 552 | """ 553 | 554 | def stop(self): 555 | """ 556 | Stops the operator driver. 557 | """ 558 | 559 | def abort(self): 560 | """ 561 | Aborts the driver so that no more callbacks can be made to the 562 | operator. The semantics of abort and stop have deliberately been 563 | separated so that code can detect an aborted driver (i.e., via the 564 | return status of OperatorMasterDriver.join), and instantiate and start 565 | another driver if desired (from within the same process, although this 566 | functionality is currently not supported for executors). 567 | """ 568 | 569 | def join(self): 570 | """ 571 | Waits for the driver to be stopped or aborted, possibly blocking the 572 | current thread indefinitely. The return status of this function can 573 | be used to determine if the driver was aborted (see mesos.proto for a 574 | description of Status). 575 | """ 576 | 577 | def run(self): 578 | """ 579 | Starts and immediately joins (i.e., blocks on) the driver. 580 | """ 581 | 582 | def getAgents(self): 583 | """ 584 | This call retrieves information about all the agents known to the master. 585 | """ 586 | 587 | def getRoles(self): 588 | """ 589 | Query the information about roles. 590 | """ 591 | 592 | def getWeights(self): 593 | """ 594 | This call retrieves the information about role weights. 595 | """ 596 | 597 | def updateWeights(self, weight_infos): 598 | """ 599 | This call updates weights for specific role. This call takes weight_infos which needs role value and weight value. 600 | """ 601 | 602 | def getMaster(self): 603 | """ 604 | This call retrieves the information on master. 605 | """ 606 | 607 | def reserveResources(self, agent_id, resources): 608 | """ 609 | This call reserve resources dynamically on a specific agent. This call takes agent_id and resources details like 610 | the following. 611 | """ 612 | 613 | def unreserveResources(self, agent_id, resources): 614 | """ 615 | This call unreserve resources dynamically on a specific agent. This call takes agent_id and resources details like 616 | the following. 617 | """ 618 | 619 | def createVolumes(self, agent_id, volumes): 620 | """ 621 | This call create persistent volumes on reserved resources. The request is forwarded asynchronously to the Mesos 622 | agent where the reserved resources are located. That asynchronous message may not be delivered or creating the 623 | volumes at the agent might fail. This call takes agent_id and volumes details like the following. 624 | """ 625 | 626 | def destroyVolumes(self, agent_id, volumes): 627 | """ 628 | This call destroys persistent volumes. The request is forwarded asynchronously to the Mesos agent where the 629 | reserved resources are located. 630 | """ 631 | 632 | def getMaintenanceStatus(self): 633 | """ 634 | This call retrieves the cluster's maintenance status. 635 | """ 636 | 637 | def getMaintenanceSchedule(self): 638 | """ 639 | This call retrieves the cluster's maintenance status. 640 | """ 641 | 642 | def updateMaintenanceSchedule(self, windows): 643 | """ 644 | This call retrieves the cluster's maintenance schedule. 645 | """ 646 | 647 | def startMaintenance(self, machines): 648 | """ 649 | This call starts the maintenance of the cluster, this would bring a set of machines down. 650 | """ 651 | 652 | def stopMaintenance(self, machines): 653 | """ 654 | Stops the maintenance of the cluster, this would bring a set of machines back up. 655 | """ 656 | 657 | def getQuota(self): 658 | """ 659 | This call retrieves the cluster's configured quotas. 660 | """ 661 | 662 | def setQuota(self, quota_request): 663 | """ 664 | This call sets the quota for resources to be used by a particular role. 665 | """ 666 | 667 | def removeQuota(self, role): 668 | """ 669 | This call removes the quota for a particular role. 670 | """ 671 | 672 | def markAgentGone(self, agent_id): 673 | """ 674 | This call can be used by operators to assert that an agent instance has failed and is never coming back 675 | (e.g., ephemeral instance from cloud provider). The master would shutdown the agent and send TASK_GONE_BY_OPERATOR 676 | updates for all the running tasks. This signal can be used by stateful frameworks to re-schedule their workloads 677 | (volumes, reservations etc.) to other agent instances. It is possible that the tasks might still be running if the 678 | operator's assertion was wrong and the agent was partitioned away from the master. The agent would be shutdown 679 | when it tries to re-register with the master when the partition heals. This call is idempotent. 680 | """ 681 | 682 | class OperatorMaster(object): 683 | """ 684 | Base class for Mesos operators. Users' operators should extend this 685 | class to get default implementations of methods they don't override. 686 | """ 687 | 688 | # def heartbeat(self): 689 | # """ 690 | # Periodically sent by the master to the subscriber according to 'Subscribed.heartbeat_interval_seconds'. If the 691 | # subscriber does not receive any events (including heartbeats) for an extended period of time (e.g., 5 x 692 | # heartbeat_interval_seconds), it is likely that the connection is lost or there is a network partition. In that 693 | # case, the subscriber should close the existing subscription connection and resubscribe using a backoff strategy. 694 | # """ 695 | 696 | def taskAdded(self, task_info): 697 | """ 698 | Invoked whenever a task has been added to the master. This can happen either when a new task launch is processed 699 | by the master or when an agent re-registers with a failed over master. 700 | """ 701 | 702 | def taskUpdated(self, task_info): 703 | """ 704 | Invoked whenever the state of the task changes in the master. This can happen when a status update is received or 705 | generated by the master. Since status updates are retried by the agent, not all status updates received by the 706 | master result in the event being sent. 707 | """ 708 | 709 | def frameworkAdded(self, framework_info): 710 | """ 711 | Sent whenever a framework becomes known to the master. This can happen when a new framework registers with the 712 | master. 713 | """ 714 | 715 | def frameworkUpdated(self, framework_info): 716 | """ 717 | Sent whenever a framework re-registers with the master upon a disconnection (network error) or upon a master 718 | failover. 719 | """ 720 | 721 | def frameworkRemoved(self, framework_info): 722 | """ 723 | Sent whenever a framework is removed. This can happen when a framework is explicitly teardown by the operator or 724 | if it fails to re-register with the master within the failover timeout. 725 | """ 726 | 727 | def agentAdded(self, agent_info): 728 | """ 729 | Sent whenever an agent becomes known to it. This can happen when an agent registered for the first time, or 730 | reregistered after a master failover. 731 | """ 732 | 733 | def agentRemoved(self, agent_id): 734 | """ 735 | Sent whenever a agent is removed. This can happen when the agent is scheduled for maintenance. (NOTE: It's 736 | possible that an agent might become active once it has been removed, i.e. if the master has gc'ed its list of 737 | known "dead" agents. See MESOS-5965 for context). 738 | """ 739 | 740 | class OperatorAgentDriver(OperatorDaemonDriver): 741 | """ 742 | This API contains all the calls accepted by the agent. 743 | """ 744 | 745 | def getContainers(self, show_nested=False, show_standalone=False): 746 | """ 747 | This call retrieves information about containers running on this agent. It contains ContainerStatus and 748 | ResourceStatistics along with some metadata of the containers. 749 | There are two knobs in the request to control the types of the containers this API will return: 750 | * show_nested: Whether to show nested containers [default: false]. 751 | * show_standalone: Whether to show standalone containers [default: false]. 752 | """ 753 | 754 | def launchNestedContainer(self, launch_nested_container): 755 | """ 756 | This call launches a nested container. Any authorized entity, including the executor itself, its tasks, or the 757 | operator can use this API to launch a nested container. 758 | """ 759 | 760 | def waitNestedContainer(self, container_id, parent_id=None): 761 | """ 762 | This call waits for a nested container to terminate or exit. Any authorized entity, including the executor itself, 763 | its tasks, or the operator can use this API to wait on a nested container. 764 | """ 765 | 766 | def killNestedContainer(self, container_id, parent_id=None): 767 | """ 768 | This call initiates the destruction of a nested container. Any authorized entity, including the executor itself, 769 | its tasks, or the operator can use this API to kill a nested container. 770 | """ 771 | 772 | def launchNestedContainerSession(self, launch_nested_container_session): 773 | """ 774 | This call launches a nested container whose lifetime is tied to the lifetime of the HTTP call establishing this 775 | connection. The STDOUT and STDERR of the nested container is streamed back to the client so long as the connection 776 | is active. 777 | """ 778 | 779 | def attachContainerInput(self, container_id): 780 | """ 781 | This call attaches to the STDIN of the primary process of a container and streams input to it. This call can only 782 | be made against containers that have been launched with an associated IOSwitchboard (i.e. nested containers 783 | launched via a LAUNCH_NESTED_CONTAINER_SESSION call or normal containers launched with a TTYInfo in their 784 | ContainerInfo). Only one ATTACH_CONTAINER_INPUT call can be active for a given container at a time. Subsequent 785 | attempts to attach will fail. 786 | 787 | The first message sent over an ATTACH_CONTAINER_INPUT stream must be of type CONTAINER_ID and contain the 788 | ContainerID of the container being attached to. Subsequent messages must be of type PROCESS_IO, but they may 789 | contain subtypes of either DATA or CONTROL. DATA messages must be of type STDIN and contain the actual data to 790 | stream to the STDIN of the container being attached to. Currently, the only valid CONTROL message sends a 791 | heartbeat to keep the connection alive. We may add more CONTROL messages in the future. 792 | """ 793 | 794 | def attachContainerOutput(self, container_id): 795 | """ 796 | This call attaches to the STDOUT and STDERR of the primary process of a container and streams its output back to 797 | the client. This call can only be made against containers that have been launched with an associated IOSwitchboard 798 | (i.e. nested containers launched via a LAUNCH_NESTED_CONTAINER_SESSION call or normal containers launched with a 799 | TTYInfo in their ContainerInfo field). Multiple ATTACH_CONTAINER_OUTPUT calls can be active for a given container 800 | at once. 801 | """ 802 | 803 | def removeNestedContainer(self, container_id, parent_id = None): 804 | """ 805 | This call triggers the removal of a nested container and its artifacts (e.g., the sandbox and runtime directories). 806 | This call can only be made against containers that have already terminated, and whose parent container has not 807 | been destroyed. Any authorized entity, including the executor itself, its tasks, or the operator can use this API 808 | call. 809 | """ 810 | 811 | def addResourceProviderConfig(self, info): 812 | """ 813 | This call launches a Local Resource Provider on the agent with the specified ResourceProviderInfo. 814 | """ 815 | 816 | def updateResourceProviderConfig(self, info): 817 | """ 818 | This call updates a Local Resource Provider on the agent with the specified ResourceProviderInfo. 819 | """ 820 | 821 | def removeResourceProviderConfig(self, type, name): 822 | """ 823 | This call terminates a given Local Resource Provider on the agent and prevents it from being launched again until 824 | the config is added back. The master and the agent will think the resource provider has disconnected, similar to 825 | agent disconnection. 826 | If there exists a task that is using the resources provided by the resource provider, its execution will not be 827 | affected. However, offer operations for the local resource provider will not be successful. In fact, if a local 828 | resource provider is disconnected, the master will rescind the offers related to that local resource provider, 829 | effectively disallowing frameworks to perform operations on the disconnected local resource provider. 830 | The local resource provider can be re-added after its removal using ADD_RESOURCE_PROVIDER_CONFIG. Note that 831 | removing a local resource provider is different than marking a local resource provider as gone, in which case the 832 | local resource provider will not be allowed to be re-added. Marking a local resource provider as gone is not yet 833 | supported. 834 | """ 835 | 836 | def pruneImages(self, excluded_images=None): 837 | """ 838 | This call triggers garbage collection for container images. This call can only be made when all running containers 839 | are launched with Mesos version 1.5 or newer. An optional list of excluded images from GC can be speficied via 840 | prune_images.excluded_images field. 841 | """ 842 | -------------------------------------------------------------------------------- /pymesos/operator_v1.py: -------------------------------------------------------------------------------- 1 | import json 2 | import logging 3 | from threading import RLock 4 | 5 | from addict import Dict 6 | from six.moves.http_client import HTTPConnection 7 | from six.moves.urllib.parse import urlparse 8 | 9 | from .interface import OperatorDaemonDriver 10 | from .process import Process 11 | from .utils import DAY 12 | 13 | logger = logging.getLogger(__name__) 14 | 15 | 16 | class MesosOperatorDaemonDriver(OperatorDaemonDriver): 17 | _timeout = 10 18 | 19 | def __init__(self, daemon_uri): 20 | self.init(daemon_uri) 21 | 22 | def init(self, daemon_uri): 23 | """ 24 | :param daemon_uri: masterHost:5050 or agentHost:5051 25 | """ 26 | self._daemon = daemon_uri 27 | self._conn = None 28 | 29 | def _get_conn(self): 30 | if self._conn is not None: 31 | return self._conn 32 | 33 | host, port = self._daemon.split(':', 2) 34 | port = int(port) 35 | self._conn = HTTPConnection(host, port, timeout=self._timeout) 36 | return self._conn 37 | 38 | def _send(self, body, path='/api/v1/operator', method='POST', headers={}): 39 | with self._lock: 40 | conn = self._get_conn() 41 | if conn is None: 42 | raise RuntimeError('Not connected yet') 43 | 44 | if body != '': 45 | data = json.dumps(body).encode('utf-8') 46 | headers['Content-Type'] = 'application/json' 47 | else: 48 | data = '' 49 | 50 | # stream_id = self.stream_id 51 | # if stream_id: 52 | # headers['Mesos-Stream-Id'] = stream_id 53 | 54 | try: 55 | conn.request(method, path, body=data, headers=headers) 56 | resp = conn.getresponse() 57 | except Exception: 58 | self._conn.close() 59 | self._conn = None 60 | raise 61 | 62 | if resp.status >= 300 and resp.status <= 399: 63 | url = resp.getheader('location') 64 | parsed = urlparse(url) 65 | self._daemon = '%s:%s' % (parsed.hostname, parsed.port) 66 | self._conn.close() 67 | self._conn = None 68 | return self._send(body, path, method, headers) 69 | 70 | if resp.status < 200 or resp.status >= 300: 71 | raise RuntimeError( 72 | 'Failed to send request code=%s, message=%s' % ( 73 | resp.status, resp.read() 74 | ) 75 | ) 76 | 77 | result = resp.read() 78 | if not result: 79 | return {} 80 | 81 | try: 82 | return json.loads(result.decode('utf-8')) 83 | except Exception: 84 | return {} 85 | 86 | def getHealth(self): 87 | body = dict( 88 | type='GET_HEALTH', 89 | ) 90 | return self._send(body) 91 | 92 | def getFlags(self): 93 | body = dict( 94 | type='GET_FLAGS', 95 | ) 96 | return self._send(body) 97 | 98 | def getVersion(self): 99 | body = dict( 100 | type='GET_VERSION', 101 | ) 102 | return self._send(body) 103 | 104 | def getMetrics(self, timeout): 105 | body = dict( 106 | type='GET_METRICS', 107 | get_metrics=dict( 108 | timeout=dict( 109 | nanoseconds=timeout, 110 | ) 111 | ) 112 | ) 113 | return self._send(body) 114 | 115 | def getLoggingLevel(self): 116 | body = dict( 117 | type='GET_LOGGING_LEVEL', 118 | ) 119 | return self._send(body) 120 | 121 | def setLoggingLevel(self, level, duration): 122 | body = dict( 123 | type='SET_LOGGING_LEVEL', 124 | set_logging_level=dict( 125 | duration=dict( 126 | nanoseconds=duration, 127 | ), 128 | level=level, 129 | ) 130 | ) 131 | return self._send(body) 132 | 133 | def listFiles(self, path): 134 | body = dict( 135 | type='LIST_FILES', 136 | list_files=dict( 137 | path=path, 138 | ), 139 | ) 140 | return self._send(body) 141 | 142 | def readFile(self, path, length, offset): 143 | body = dict( 144 | type='READ_FILE', 145 | read_file=dict( 146 | length=length, 147 | offset=offset, 148 | path=path, 149 | ) 150 | ) 151 | return self._send(body) 152 | 153 | def getState(self): 154 | body = dict( 155 | type='GET_STATE', 156 | ) 157 | return self._send(body) 158 | 159 | def getFrameworks(self): 160 | body = dict( 161 | type='GET_FRAMEWORKS', 162 | ) 163 | return self._send(body) 164 | 165 | def getExecutors(self): 166 | body = dict( 167 | type='GET_EXECUTORS', 168 | ) 169 | return self._send(body) 170 | 171 | def getTasks(self): 172 | body = dict( 173 | type='GET_TASKS', 174 | ) 175 | return self._send(body) 176 | 177 | 178 | class MesosOperatorMasterDriver(Process, MesosOperatorDaemonDriver): 179 | def __init__(self, master_uri, operator=None, use_addict=False, 180 | timeout=DAY): 181 | """ 182 | 183 | :param master_uri: 184 | :param operator: Optional. Only if you want to send requests that 185 | result in a stream of events (SUBSCRIBE). 186 | :type operator: OperatorMaster 187 | """ 188 | super(MesosOperatorMasterDriver, self).__init__(master=master_uri, 189 | timeout=timeout) 190 | super(MesosOperatorMasterDriver, self).init(master_uri) 191 | self.operator = operator 192 | self.master_uri = master_uri 193 | self._dict_cls = Dict if use_addict else dict 194 | 195 | def start(self): 196 | super(MesosOperatorMasterDriver, self).start() 197 | uri = self.master_uri 198 | if uri.startswith('zk://') or uri.startswith('zoo://'): 199 | from .detector import MasterDetector 200 | self.detector = MasterDetector(uri[uri.index('://') + 3:], self) 201 | self.detector.start() 202 | else: 203 | if ':' not in uri: 204 | uri += ':5050' 205 | self.change_master(uri) 206 | 207 | def gen_request(self): 208 | body = json.dumps( 209 | dict( 210 | type='SUBSCRIBE', 211 | ), 212 | ) 213 | 214 | request = ('POST /api/v1/operator HTTP/1.1\r\nHost: %s\r\n' 215 | 'Content-Type: application/json\r\n' 216 | 'Accept: application/json\r\n' 217 | 'Connection: close\r\nContent-Length: %s\r\n\r\n%s') % ( 218 | self.master, len(body), body 219 | ) 220 | return request.encode('utf-8') 221 | 222 | def on_event(self, event): 223 | if 'type' in event: 224 | _type = event['type'].lower() 225 | if _type == 'heartbeat': 226 | return 227 | 228 | if _type not in event: 229 | logger.error( 230 | 'Missing `%s` in event %s' % 231 | (_type, event)) 232 | return 233 | 234 | event = event[_type] 235 | func_name = 'on_%s' % (_type,) 236 | func = getattr(self, func_name, None) 237 | if func is not None: 238 | func(event) 239 | else: 240 | logger.error('Unknown type:%s, event:%s' % (_type, event)) 241 | else: 242 | logger.error('Unknown event:%s' % (event,)) 243 | 244 | def on_close(self): 245 | if self._conn is not None: 246 | self._conn.close() 247 | self._conn = None 248 | 249 | def on_subscribed(self, info): 250 | state = info['get_state'] 251 | logger.info( 252 | 'Operator client subscribed with cluster state: %s' % state) 253 | 254 | def on_task_added(self, event): 255 | task_info = event['task'] 256 | self.operator.taskAdded(self._dict_cls(task_info)) 257 | 258 | def on_task_updated(self, event): 259 | self.operator.taskUpdated(self._dict_cls(event)) 260 | 261 | def on_framework_added(self, event): 262 | framework_info = event['framework'] 263 | self.operator.frameworkAdded(self._dict_cls(framework_info)) 264 | 265 | def on_framework_updated(self, event): 266 | framework_info = event['framework'] 267 | self.operator.frameworkUpdated(self._dict_cls(framework_info)) 268 | 269 | def on_framework_removed(self, event): 270 | framework_info = event['framework_info'] 271 | self.operator.frameworkRemoved(self._dict_cls(framework_info)) 272 | 273 | def on_agent_added(self, event): 274 | agent_info = event['agent'] 275 | self.operator.agentAdded(self._dict_cls(agent_info)) 276 | 277 | def on_agent_removed(self, event): 278 | agent_id = event['agent_id']['value'] 279 | self.operator.agentRemoved(agent_id) 280 | 281 | def getAgents(self): 282 | body = dict( 283 | type='GET_AGENTS', 284 | ) 285 | return self._send(body) 286 | 287 | def getRoles(self): 288 | body = dict( 289 | type='GET_ROLES', 290 | ) 291 | return self._send(body) 292 | 293 | def getWeights(self): 294 | body = dict( 295 | type='GET_WEIGHTS', 296 | ) 297 | return self._send(body) 298 | 299 | def updateWeights(self, weight_infos): 300 | body = dict( 301 | type='UPDATE_WEIGHTS', 302 | update_weights=dict( 303 | weight_infos=[dict(role=weight_info['role'], 304 | weight=weight_info['weight']) for 305 | weight_info in 306 | weight_infos], 307 | ), 308 | ) 309 | self._send(body) 310 | 311 | def getMaster(self): 312 | body = dict( 313 | type='GET_MASTER', 314 | ) 315 | return self._send(body) 316 | 317 | def reserveResources(self, agent_id, resources): 318 | body = dict( 319 | type='RESERVE_RESOURCES', 320 | reserve_resources=dict( 321 | agent_id=dict(value=agent_id), 322 | resources=resources, 323 | ), 324 | ) 325 | self._send(body) 326 | 327 | def unreserveResources(self, agent_id, resources): 328 | body = dict( 329 | type='UNRESERVE_RESOURCES', 330 | unreserve_resources=dict( 331 | agent_id=dict(value=agent_id), 332 | resources=resources, 333 | ), 334 | ) 335 | self._send(body) 336 | 337 | def createVolumes(self, agent_id, volumes): 338 | body = dict( 339 | type='CREATE_VOLUMES', 340 | create_volumes=dict( 341 | agent_id=dict(value=agent_id), 342 | volumes=volumes, 343 | ), 344 | ) 345 | self._send(body) 346 | 347 | def destroyVolumes(self, agent_id, volumes): 348 | body = dict( 349 | type='DESTROY_VOLUMES', 350 | destroy_volumes=dict( 351 | agent_id=dict(value=agent_id), 352 | volumes=volumes, 353 | ), 354 | ) 355 | self._send(body) 356 | 357 | def getMaintenanceStatus(self): 358 | body = dict( 359 | type='GET_MAINTENANCE_STATUS', 360 | ) 361 | return self._send(body) 362 | 363 | def getMaintenanceSchedule(self): 364 | body = dict( 365 | type='GET_MAINTENANCE_SCHEDULE', 366 | ) 367 | return self._send(body) 368 | 369 | def updateMaintenanceSchedule(self, windows): 370 | body = dict( 371 | type='UPDATE_MAINTENANCE_SCHEDULE', 372 | update_maintenance_schedule=dict( 373 | schedule=dict( 374 | windows=windows 375 | ), 376 | ), 377 | ) 378 | self._send(body) 379 | 380 | def startMaintenance(self, machines): 381 | body = dict( 382 | type='START_MAINTENANCE', 383 | start_maintenance=dict( 384 | machines=[dict( 385 | hostname=machine['hostname'], ip=machine['ip']) for machine 386 | in machines], 387 | ), 388 | ) 389 | self._send(body) 390 | 391 | def stopMaintenance(self, machines): 392 | body = dict( 393 | type='STOP_MAINTENANCE', 394 | stop_maintenance=dict( 395 | machines=[dict( 396 | hostname=machine['hostname'], ip=machine['ip']) for machine 397 | in machines], 398 | ), 399 | ) 400 | self._send(body) 401 | 402 | def getQuota(self): 403 | body = dict( 404 | type='GET_QUOTA', 405 | ) 406 | return self._send(body) 407 | 408 | def setQuota(self, quota_request): 409 | body = dict( 410 | type='SET_QUOTA', 411 | set_quota=dict( 412 | quota_request=quota_request, 413 | ), 414 | ) 415 | self._send(body) 416 | 417 | def removeQuota(self, role): 418 | body = dict( 419 | type='REMOVE_QUOTA', 420 | remove_quota=dict( 421 | role=role, 422 | ), 423 | ) 424 | self._send(body) 425 | 426 | def markAgentGone(self, agent_id): 427 | body = dict( 428 | type='MARK_AGENT_GONE', 429 | mark_agent_gone=dict( 430 | agent_id=dict( 431 | value=agent_id, 432 | ), 433 | ), 434 | ) 435 | self._send(body) 436 | 437 | 438 | class MesosOperatorAgentDriver(MesosOperatorDaemonDriver): 439 | def __init__(self, agent_uri): 440 | super(MesosOperatorAgentDriver, self).__init__(agent_uri) 441 | self._lock = RLock() 442 | 443 | def getContainers(self, show_nested=False, show_standalone=False): 444 | body = dict( 445 | type='GET_CONTAINERS', 446 | get_containers=dict( 447 | show_nested=show_nested, 448 | show_standalone=show_standalone, 449 | ), 450 | ) 451 | return self._send(body) 452 | 453 | def launchNestedContainer(self, launch_nested_container): 454 | body = dict( 455 | type='LAUNCH_NESTED_CONTAINER', 456 | launch_nested_container=launch_nested_container, 457 | ) 458 | self._send(body) 459 | 460 | def waitNestedContainer(self, container_id, parent_id=None): 461 | body = dict( 462 | type='WAIT_NESTED_CONTAINER', 463 | wait_nested_container=dict( 464 | container_id=dict( 465 | value=container_id, 466 | ), 467 | ), 468 | ) 469 | if parent_id is not None: 470 | body['wait_nested_container']['container_id']['parent'] = dict( 471 | value=parent_id, ) 472 | self._send(body) 473 | 474 | def killNestedContainer(self, container_id, parent_id=None): 475 | body = dict( 476 | type='KILL_NESTED_CONTAINER', 477 | kill_nested_container=dict( 478 | container_id=dict( 479 | value=container_id, 480 | ), 481 | ), 482 | ) 483 | if parent_id is not None: 484 | body['kill_nested_container']['container_id']['parent'] = dict( 485 | value=parent_id, ) 486 | self._send(body) 487 | 488 | def launchNestedContainerSession(self, launch_nested_container_session): 489 | body = dict( 490 | type='LAUNCH_NESTED_CONTAINER_SESSION', 491 | launch_nested_container_session=launch_nested_container_session, 492 | ) 493 | headers = {'Accept': 'application/recordio'} 494 | self._send(body, headers=headers) 495 | 496 | def attachContainerInput(self, container_id, process_ios): 497 | # The first message sent over an ATTACH_CONTAINER_INPUT stream must be 498 | # of type CONTAINER_ID and contain the ContainerID of the container 499 | # being attached to. 500 | msg = dict( 501 | type='ATTACH_CONTAINER_INPUT', 502 | attach_container_input=dict( 503 | type='CONTAINER_ID', 504 | container_id=dict( 505 | value=container_id, 506 | ) 507 | ), 508 | ) 509 | # Messages are encoded in RecordIO format 510 | sort_keys = True 511 | msg_str = json.dumps(msg, sort_keys=sort_keys) 512 | record_io = '{}\n{}'.format(len(msg_str), msg_str) 513 | body = '' 514 | body += record_io 515 | # Subsequent messages must be of type PROCESS_IO 516 | # Template for PROCESS_IO messages 517 | process_io = dict( 518 | type='ATTACH_CONTAINER_INPUT', 519 | attach_container_input=dict( 520 | type='PROCESS_IO', 521 | process_io='DUMMY', 522 | ) 523 | ) 524 | for msg in process_ios: 525 | if msg['type'] != 'DATA' and msg['type'] != 'CONTROL': 526 | raise ValueError( 527 | 'PROCESS_IO messages may contain subtypes of either DATA \ 528 | or CONTROL') 529 | process_io['attach_container_input']['process_io'] = msg 530 | msg_str = json.dumps(process_io, sort_keys=sort_keys) 531 | record_io = '{}\n{}'.format(len(msg_str), msg_str) 532 | body += record_io 533 | headers = {'Accept': 'application/recordio'} 534 | self._send(body, headers=headers) 535 | 536 | def attachContainerOutput(self, container_id): 537 | body = dict( 538 | type='ATTACH_CONTAINER_OUTPUT', 539 | attach_container_output=dict( 540 | container_id=dict( 541 | value=container_id, 542 | ) 543 | ), 544 | ) 545 | headers = {'Accept': 'application/recordio', 546 | 'Message-Accept': 'application/json'} 547 | return self._send(body, headers=headers) 548 | 549 | def removeNestedContainer(self, container_id, parent_id=None): 550 | body = dict( 551 | type='REMOVE_NESTED_CONTAINER', 552 | remove_nested_container=dict( 553 | container_id=dict( 554 | value=container_id, 555 | ), 556 | ), 557 | ) 558 | if parent_id is not None: 559 | body['remove_nested_container']['container_id']['parent'] = dict( 560 | value=parent_id, ) 561 | self._send(body) 562 | 563 | def addResourceProviderConfig(self, info): 564 | body = dict( 565 | type='ADD_RESOURCE_PROVIDER_CONFIG', 566 | add_resource_provider_config=dict( 567 | info=info, 568 | ), 569 | ) 570 | self._send(body) 571 | 572 | def updateResourceProviderConfig(self, info): 573 | body = dict( 574 | type='UPDATE_RESOURCE_PROVIDER_CONFIG', 575 | update_resource_provider_config=dict( 576 | info=info, 577 | ), 578 | ) 579 | self._send(body) 580 | 581 | def removeResourceProviderConfig(self, type, name): 582 | body = dict( 583 | type='REMOVE_RESOURCE_PROVIDER_CONFIG', 584 | remove_resource_provider_config=dict( 585 | type=type, 586 | name=name, 587 | ), 588 | ) 589 | self._send(body) 590 | 591 | def pruneImages(self, excluded_images=None): 592 | body = dict( 593 | type='PRUNE_IMAGES', 594 | ) 595 | if excluded_images is not None: 596 | body['prune_images'] = dict( 597 | excluded_images=excluded_images, 598 | ) 599 | self._send(body) 600 | -------------------------------------------------------------------------------- /pymesos/process.py: -------------------------------------------------------------------------------- 1 | import os 2 | import re 3 | import sys 4 | import json 5 | import time 6 | import math 7 | import errno 8 | import random 9 | import select 10 | import signal 11 | import socket 12 | import logging 13 | from six import reraise 14 | from six.moves import _thread as thread 15 | from six.moves.http_client import OK, TEMPORARY_REDIRECT, SERVICE_UNAVAILABLE 16 | from six.moves.urllib.parse import urlparse 17 | from threading import Thread, RLock 18 | from http_parser.http import HttpParser 19 | from .utils import DAY 20 | 21 | 22 | def _strerror(err): 23 | try: 24 | return os.strerror(err) 25 | except (ValueError, OverflowError, NameError): 26 | if err in errno.errorcode: 27 | return errno.errorcode[err] 28 | return "Unknown error %s" % err 29 | 30 | 31 | def _handle_sigint(signum, frame): 32 | global _prev_handler, _exc_info 33 | assert signum == signal.SIGINT 34 | if _exc_info is not None: 35 | exc_info = _exc_info 36 | _exc_info = None 37 | reraise(*exc_info) 38 | elif _prev_handler is not None: 39 | return _prev_handler(signum, frame) 40 | 41 | raise KeyboardInterrupt 42 | 43 | 44 | _exc_info = None 45 | _prev_handler = signal.signal(signal.SIGINT, _handle_sigint) 46 | LENGTH_PATTERN = re.compile(br'\d+\n') 47 | logger = logging.getLogger(__name__) 48 | PIPE_BUF = getattr(select, 'PIPE_BUF', 4096) 49 | 50 | SLEEP_TIMEOUT_SCALE = 2 51 | SLEEP_TIMEOUT_INIT = 2 52 | # SLEEP_TIMEOUT_MAX used by backoff and cater to mesos restriction 53 | # https://github.com/apache/mesos/blob/0dc24dba9afdca1948eea5cad05f861162dc8dd6/src/slave/constants.hpp#L45 54 | SLEEP_TIMEOUT_MAX = 15 55 | SELECT_TIMEOUT = 2 56 | 57 | 58 | class Connection(object): 59 | 60 | def __init__(self, addr, callback): 61 | host, port = addr.split(':', 2) 62 | port = int(port) 63 | self._addr = (host, port) 64 | self._sock = socket.socket() 65 | self._sock.setblocking(0) 66 | self.connected = False 67 | try: 68 | self._sock.connect(self._addr) 69 | except socket.error as e: 70 | if e.errno != errno.EAGAIN and e.errno != errno.EINPROGRESS: 71 | raise 72 | self._parser = HttpParser() 73 | self._callback = callback 74 | self._stream_id = None 75 | self._request = callback.gen_request() 76 | self._response = b'' 77 | 78 | @property 79 | def addr(self): 80 | return self._addr 81 | 82 | @property 83 | def stream_id(self): 84 | return self._stream_id 85 | 86 | def handle_connect_event(self): 87 | err = self._sock.getsockopt(socket.SOL_SOCKET, socket.SO_ERROR) 88 | if err == 0: 89 | self.connected = True 90 | return "" 91 | return _strerror(err) 92 | 93 | def write(self): 94 | try: 95 | sent = self._sock.send(self._request) 96 | self._request = self._request[sent:] 97 | return True 98 | except socket.error as e: 99 | if e.errno == errno.EAGAIN: 100 | return True 101 | 102 | logger.exception('Failed to send to %s', self._addr) 103 | return False 104 | 105 | def read(self): 106 | try: 107 | buf = self._sock.recv(PIPE_BUF) 108 | n_recv = len(buf) 109 | if n_recv == 0: 110 | logger.error('Remote %s closed', self.addr) 111 | return False 112 | 113 | n_parsed = self._parser.execute(buf, n_recv) 114 | if n_parsed != n_recv: 115 | if hasattr(self._parser, 'errno'): 116 | # using http_parser.pyparser 117 | 118 | from http_parser.pyparser import INVALID_CHUNK 119 | if self._parser.errno == INVALID_CHUNK: 120 | # need more chunk data 121 | return True 122 | 123 | if self._parser.errno: 124 | raise RuntimeError( 125 | 'Failed to parse, code:%s %s' % ( 126 | self._parser.errno, 127 | self._parser.errstr, 128 | ) 129 | ) 130 | 131 | else: 132 | raise RuntimeError( 133 | 'Failed to parse, code:%s' % ( 134 | self._parser.get_errno(), 135 | ) 136 | ) 137 | 138 | if self._stream_id is None and self._parser.is_headers_complete(): 139 | code = self._parser.get_status_code() 140 | if code == TEMPORARY_REDIRECT: 141 | headers = { 142 | k.upper(): v 143 | for k, v in list(self._parser.get_headers().items()) 144 | } 145 | new_master = headers['LOCATION'] 146 | new_master = urlparse(new_master).netloc or new_master 147 | logger.warning( 148 | 'Try to redirect to new master: %s', new_master 149 | ) 150 | self._callback.change_master(new_master) 151 | return False 152 | 153 | elif code == SERVICE_UNAVAILABLE: 154 | logger.warning('Master is not available, retry.') 155 | return False 156 | 157 | elif code != OK: 158 | msg = self._parser.recv_body() 159 | if not self._parser.is_message_complete(): 160 | msg += ' ...' 161 | 162 | raise RuntimeError('Failed with HTTP %s: %s' % (code, msg)) 163 | if not self._parser.is_chunked(): 164 | raise RuntimeError('Response is not chunked') 165 | 166 | headers = { 167 | k.upper(): v 168 | for k, v in list(self._parser.get_headers().items()) 169 | } 170 | self._stream_id = headers.get('MESOS-STREAM-ID', '') 171 | self._callback.stream_id = self._stream_id 172 | 173 | if self._parser.is_partial_body(): 174 | self._response += self._parser.recv_body() 175 | while True: 176 | m = LENGTH_PATTERN.match(self._response) 177 | if not m: 178 | break 179 | 180 | captured = m.group(0) 181 | length = int(captured.strip()) 182 | if len(self._response) < len(captured) + length: 183 | break 184 | 185 | data = self._response[ 186 | len(captured):len(captured) + length] 187 | self._response = self._response[ 188 | len(captured) + length:] 189 | try: 190 | event = json.loads(data.decode('utf-8')) 191 | except Exception: 192 | logger.exception('Failed parse json %s', data) 193 | raise 194 | 195 | try: 196 | self._callback.process_event(event) 197 | except Exception: 198 | logger.exception('Failed to process event') 199 | raise 200 | 201 | if self._parser.is_message_complete(): 202 | logger.debug('Event stream ended') 203 | return False 204 | 205 | return True 206 | except socket.error as e: 207 | if e.errno == errno.EAGAIN: 208 | return True 209 | 210 | logger.exception('Failed to recv from %s', self._addr) 211 | return False 212 | 213 | def want_write(self): 214 | return bool(self._request) 215 | 216 | def fileno(self): 217 | return self._sock.fileno() 218 | 219 | def close(self): 220 | self._sock.close() 221 | self._sock = None 222 | self.connected = False 223 | self._parser = None 224 | self._request = None 225 | self._response = None 226 | self._callback.on_close() 227 | 228 | 229 | class Process(object): 230 | 231 | def __init__(self, master=None, timeout=DAY): 232 | self._master = None 233 | self._started = False 234 | self._lock = RLock() 235 | self._wakeup_fds = None 236 | self._io_thread = None 237 | self._new_master = master 238 | self._stream_id = None 239 | self._timeout = timeout 240 | 241 | @property 242 | def aborted(self): 243 | with self._lock: 244 | return not self._started 245 | 246 | @property 247 | def master(self): 248 | with self._lock: 249 | return self._master 250 | 251 | @property 252 | def stream_id(self): 253 | with self._lock: 254 | return self._stream_id 255 | 256 | @stream_id.setter 257 | def stream_id(self, _stream_id): 258 | with self._lock: 259 | self._stream_id = _stream_id 260 | 261 | @property 262 | def connected(self): 263 | return self.stream_id is not None 264 | 265 | def gen_request(self): 266 | raise NotImplementedError 267 | 268 | def on_event(self, event): 269 | raise NotImplementedError 270 | 271 | def on_close(self): 272 | raise NotImplementedError 273 | 274 | def process_event(self, event): 275 | if self._started: 276 | self.on_event(event) 277 | 278 | def change_master(self, new_master): 279 | with self._lock: 280 | self._new_master = new_master 281 | 282 | self._notify() 283 | 284 | def _notify(self): 285 | with self._lock: 286 | if self._wakeup_fds: 287 | os.write(self._wakeup_fds[1], b'\0') 288 | 289 | def _shutdown(self): 290 | pass 291 | 292 | def _run(self): 293 | try: 294 | with self._lock: 295 | _wakeup_fd = self._wakeup_fds[0] 296 | 297 | conn = None 298 | self.stream_id = None 299 | num_conn_retry = 0 300 | connect_deadline = 0 301 | 302 | while True: 303 | if not conn and self._master: 304 | conn = Connection(self._master, self) 305 | 306 | # reset deadline at first retry of every period 307 | if num_conn_retry < 1: 308 | connect_deadline = self._timeout + time.time() 309 | 310 | if time.time() > connect_deadline: 311 | raise Exception("connect reach deadline") 312 | 313 | to_write = set() 314 | to_read = set([_wakeup_fd]) 315 | if conn: 316 | to_read.add(conn.fileno()) 317 | if conn.want_write(): 318 | to_write.add(conn.fileno()) 319 | 320 | readable, writeable, _ = select.select(to_read, to_write, [], 321 | SELECT_TIMEOUT) 322 | 323 | if _wakeup_fd in readable: 324 | with self._lock: 325 | if not self._started: 326 | break 327 | 328 | if self._new_master != self._master: 329 | if conn is not None: 330 | to_read.discard(conn.fileno()) 331 | to_write.discard(conn.fileno()) 332 | conn.close() 333 | conn = None 334 | self.stream_id = None 335 | self._master = self._new_master 336 | num_conn_retry = 0 337 | 338 | readable.remove(_wakeup_fd) 339 | os.read(_wakeup_fd, PIPE_BUF) 340 | 341 | if not conn: 342 | continue 343 | 344 | if not conn.connected: 345 | err = conn.handle_connect_event() 346 | if err: 347 | sleep_timeout = self._backoff(num_conn_retry) 348 | num_conn_retry += 1 349 | conn.close() 350 | conn = None 351 | 352 | deadline = time.strftime('%Y-%m-%d %H:%M:%S', 353 | time.localtime( 354 | connect_deadline)) 355 | logger.warning( 356 | 'connect to %s error: %s,' 357 | ' sleep %ds and try again,' 358 | ' deadline: %s', 359 | self._master, err, sleep_timeout, deadline) 360 | 361 | time.sleep(sleep_timeout) 362 | continue 363 | else: 364 | num_conn_retry = 0 365 | 366 | if writeable: 367 | if not conn.write(): 368 | conn.close() 369 | conn = None 370 | self.stream_id = None 371 | continue 372 | 373 | if readable: 374 | if not conn.read(): 375 | conn.close() 376 | conn = None 377 | self.stream_id = None 378 | except Exception: 379 | logger.exception('Thread abort:') 380 | with self._lock: 381 | self._started = False 382 | 383 | global _exc_info 384 | _exc_info = sys.exc_info() 385 | thread.interrupt_main() 386 | 387 | finally: 388 | self._shutdown() 389 | 390 | if conn: 391 | conn.close() 392 | conn = None 393 | 394 | with self._lock: 395 | r, w = self._wakeup_fds 396 | os.close(r) 397 | os.close(w) 398 | self._wakeup_fds = None 399 | 400 | def _backoff(self, num_conn_retry): 401 | def _random_time(new, old): 402 | max_shift = (new - old) / SLEEP_TIMEOUT_SCALE 403 | return random.uniform(-max_shift, max_shift) 404 | 405 | new_timeout = SLEEP_TIMEOUT_INIT * math.pow(SLEEP_TIMEOUT_SCALE, 406 | num_conn_retry) 407 | old_timeout = new_timeout / SLEEP_TIMEOUT_SCALE 408 | 409 | return min(SLEEP_TIMEOUT_MAX, 410 | new_timeout + _random_time(new_timeout, old_timeout)) 411 | 412 | def start(self): 413 | with self._lock: 414 | if self._started: 415 | logger.warning('Process already started!') 416 | return 417 | 418 | if self._io_thread: 419 | self.join() 420 | 421 | self._wakeup_fds = os.pipe() 422 | self._started = True 423 | self._io_thread = Thread(target=self._run, name='Process IO') 424 | self._io_thread.daemon = True 425 | self._io_thread.start() 426 | 427 | def abort(self): 428 | self.stop() 429 | 430 | def stop(self): 431 | with self._lock: 432 | self._started = False 433 | 434 | self._notify() 435 | 436 | def join(self): 437 | if self._io_thread: 438 | self._io_thread.join() 439 | self._io_thread = None 440 | 441 | def run(self): 442 | self.start() 443 | self.join() 444 | -------------------------------------------------------------------------------- /pymesos/scheduler.py: -------------------------------------------------------------------------------- 1 | import six 2 | import json 3 | import logging 4 | from addict import Dict 5 | from six.moves.http_client import HTTPConnection 6 | from binascii import b2a_base64 7 | from .process import Process 8 | from .interface import SchedulerDriver 9 | from .utils import DAY 10 | 11 | logger = logging.getLogger(__name__) 12 | 13 | 14 | class MesosSchedulerDriver(Process, SchedulerDriver): 15 | _timeout = 10 16 | 17 | def __init__(self, sched, framework, master_uri, 18 | use_addict=False, implicit_acknowledgements=True, 19 | principal=None, secret=None, failover=False, 20 | timeout=DAY): 21 | super(MesosSchedulerDriver, self).__init__(timeout=timeout) 22 | self.sched = sched 23 | self.master_uri = master_uri 24 | self._framework = framework 25 | self.detector = None 26 | self._conn = None 27 | self.version = None 28 | self._failover = failover 29 | self._dict_cls = Dict if use_addict else dict 30 | self.implicit_acknowledgements = implicit_acknowledgements 31 | if principal is not None and secret is not None: 32 | self._basic_credential = 'Basic %s' % ( 33 | b2a_base64( 34 | ('%s:%s' % (principal, secret)).encode('ascii') 35 | ).decode('ascii').strip() 36 | ) 37 | else: 38 | self._basic_credential = None 39 | 40 | @property 41 | def framework(self): 42 | framework = dict(self._framework) 43 | version = self.version and tuple( 44 | int(n) for n in self.version.split('.') 45 | ) 46 | 47 | capabilities = [ 48 | c for c in framework.get('capabilities', []) 49 | if c['type'] != 'GPU_RESOURCES' 50 | ] 51 | 52 | if version and version >= (1, 0, 0): 53 | capabilities.append(dict(type='GPU_RESOURCES')) 54 | 55 | if capabilities: 56 | framework['capabilities'] = capabilities 57 | else: 58 | framework.pop('capabilities', None) 59 | 60 | if 'failover_timeout' not in framework: 61 | framework['failover_timeout'] = 100 62 | 63 | return framework 64 | 65 | @property 66 | def framework_id(self): 67 | id = self._framework.get('id') 68 | return id and id.get('value') 69 | 70 | @framework_id.setter 71 | def framework_id(self, id): 72 | self._framework['id'] = dict(value=id) 73 | 74 | def _get_version(self, master): 75 | if master is not None: 76 | conn = None 77 | host, port = master.split(':', 2) 78 | port = int(port) 79 | try: 80 | conn = HTTPConnection(host, port, timeout=self._timeout) 81 | conn.request('GET', '/version') 82 | resp = conn.getresponse() 83 | if resp.status < 200 or resp.status >= 300: 84 | return 85 | 86 | return json.loads(resp.read().decode('utf-8'))['version'] 87 | except Exception: 88 | logger.exception('Error') 89 | pass 90 | finally: 91 | if conn: 92 | conn.close() 93 | 94 | def change_master(self, master): 95 | self.version = self._get_version(master) 96 | super(MesosSchedulerDriver, self).change_master(master) 97 | self._close() 98 | 99 | def start(self): 100 | super(MesosSchedulerDriver, self).start() 101 | uri = self.master_uri 102 | if uri.startswith('zk://') or uri.startswith('zoo://'): 103 | from .detector import MasterDetector 104 | self.detector = MasterDetector(uri[uri.index('://') + 3:], self) 105 | self.detector.start() 106 | else: 107 | if ':' not in uri: 108 | uri += ':5050' 109 | self.change_master(uri) 110 | 111 | def stop(self, failover=False): 112 | with self._lock: 113 | self._failover = failover 114 | detector = self.detector 115 | self.detector = None 116 | 117 | if detector: 118 | detector.stop() 119 | 120 | super(MesosSchedulerDriver, self).stop() 121 | 122 | def _shutdown(self): 123 | if not self._failover: 124 | try: 125 | self._teardown() 126 | except Exception: 127 | logger.exception('Failed to Teardown') 128 | 129 | def _get_conn(self): 130 | if not self.connected: 131 | return None 132 | 133 | if self._conn is not None: 134 | return self._conn 135 | 136 | host, port = self.master.split(':', 2) 137 | port = int(port) 138 | self._conn = HTTPConnection(host, port, timeout=self._timeout) 139 | return self._conn 140 | 141 | def _send(self, body, path='/api/v1/scheduler', method='POST', headers={}): 142 | with self._lock: 143 | conn = self._get_conn() 144 | if conn is None: 145 | raise RuntimeError('Not connected yet') 146 | 147 | if body != '': 148 | data = json.dumps(body).encode('utf-8') 149 | headers['Content-Type'] = 'application/json' 150 | else: 151 | data = b'' 152 | 153 | stream_id = self.stream_id 154 | if stream_id: 155 | headers['Mesos-Stream-Id'] = stream_id 156 | 157 | if self._basic_credential: 158 | headers['Authorization'] = self._basic_credential 159 | 160 | try: 161 | conn.request(method, path, body=data, headers=headers) 162 | resp = conn.getresponse() 163 | except Exception: 164 | self._close() 165 | raise 166 | 167 | if resp.status < 200 or resp.status >= 300: 168 | raise RuntimeError('Failed to send request %s: %s\n%s' % ( 169 | resp.status, resp.read(), data)) 170 | 171 | result = resp.read() 172 | if not result: 173 | return {} 174 | 175 | try: 176 | return json.loads(result.decode('utf-8')) 177 | except Exception: 178 | return {} 179 | 180 | def _teardown(self): 181 | if self.connected: 182 | framework_id = self.framework_id 183 | if framework_id: 184 | self._send(dict( 185 | type='TEARDOWN', 186 | framework_id=dict( 187 | value=framework_id, 188 | ), 189 | )) 190 | self._framework.pop('id', None) 191 | 192 | def acceptOffers(self, offer_ids, operations, filters=None): 193 | if not operations: 194 | return self.declineOffer(offer_ids, filters=filters) 195 | 196 | if not self.connected: 197 | return 198 | 199 | framework_id = self.framework_id 200 | assert framework_id 201 | 202 | accept = dict( 203 | offer_ids=[offer_ids] 204 | if isinstance(offer_ids, dict) 205 | else offer_ids, 206 | operations=operations, 207 | ) 208 | 209 | if filters is not None: 210 | accept['filters'] = filters 211 | 212 | body = dict( 213 | type='ACCEPT', 214 | framework_id=dict( 215 | value=framework_id, 216 | ), 217 | accept=accept, 218 | ) 219 | self._send(body) 220 | 221 | def acceptInverseOffers(self, offer_ids, filters=None): 222 | framework_id = self.framework_id 223 | assert framework_id 224 | 225 | accept_inverse_offers = dict( 226 | inverse_offer_ids=[offer_ids] 227 | if isinstance(offer_ids, dict) 228 | else offer_ids 229 | ) 230 | 231 | if filters is not None: 232 | accept_inverse_offers['filters'] = filters 233 | 234 | body = dict( 235 | type='ACCEPT_INVERSE_OFFERS', 236 | framework_id=dict( 237 | value=framework_id, 238 | ), 239 | accept_inverse_offers=accept_inverse_offers, 240 | ) 241 | self._send(body) 242 | 243 | def launchTasks(self, offer_ids, tasks, filters=None): 244 | if not tasks: 245 | return self.declineOffer(offer_ids, filters=filters) 246 | 247 | if not self.connected: 248 | return 249 | 250 | framework_id = self.framework_id 251 | assert framework_id 252 | 253 | operations = [dict( 254 | type='LAUNCH', 255 | launch=dict( 256 | task_infos=tasks 257 | ), 258 | )] 259 | 260 | self.acceptOffers(offer_ids, operations, filters=filters) 261 | 262 | def declineOffer(self, offer_ids, filters=None): 263 | if not self.connected: 264 | return 265 | 266 | framework_id = self.framework_id 267 | assert framework_id 268 | decline = dict( 269 | offer_ids=[offer_ids] if isinstance(offer_ids, dict) else offer_ids 270 | ) 271 | 272 | if filters is not None: 273 | decline['filters'] = filters 274 | 275 | body = dict( 276 | type='DECLINE', 277 | framework_id=dict( 278 | value=framework_id, 279 | ), 280 | decline=decline, 281 | ) 282 | self._send(body) 283 | 284 | def declineInverseOffer(self, offer_ids, filters=None): 285 | if not self.connected: 286 | return 287 | 288 | framework_id = self.framework_id 289 | assert framework_id 290 | decline_inverse_offers = dict( 291 | inverse_offer_ids=[offer_ids] 292 | if isinstance(offer_ids, dict) 293 | else offer_ids 294 | ) 295 | 296 | if filters is not None: 297 | decline_inverse_offers['filters'] = filters 298 | 299 | body = dict( 300 | type='DECLINE_INVERSE_OFFERS', 301 | framework_id=dict( 302 | value=framework_id, 303 | ), 304 | decline_inverse_offers=decline_inverse_offers, 305 | ) 306 | self._send(body) 307 | 308 | def reviveOffers(self, roles=()): 309 | if not self.connected: 310 | return 311 | 312 | framework_id = self.framework_id 313 | assert framework_id 314 | body = dict( 315 | type='REVIVE', 316 | framework_id=dict( 317 | value=framework_id, 318 | ), 319 | ) 320 | if roles: 321 | body['revive'] = dict(roles=list(roles)) 322 | self._send(body) 323 | 324 | def suppressOffers(self, roles=()): 325 | if not self.connected: 326 | return 327 | 328 | framework_id = self.framework_id 329 | assert framework_id 330 | body = dict( 331 | type='SUPPRESS', 332 | framework_id=dict( 333 | value=framework_id, 334 | ), 335 | ) 336 | if roles: 337 | body['suppress'] = dict(roles=list(roles)) 338 | self._send(body) 339 | 340 | def killTask(self, task_id): 341 | if not self.connected: 342 | return 343 | 344 | framework_id = self.framework_id 345 | assert framework_id 346 | body = dict( 347 | type='KILL', 348 | framework_id=dict( 349 | value=framework_id, 350 | ), 351 | kill=dict( 352 | task_id=task_id, 353 | ), 354 | ) 355 | self._send(body) 356 | 357 | def acknowledgeStatusUpdate(self, status): 358 | if self.connected and 'uuid' in status: 359 | framework_id = self.framework_id 360 | assert framework_id 361 | acknowledge = dict() 362 | acknowledge['agent_id'] = status['agent_id'] 363 | acknowledge['task_id'] = status['task_id'] 364 | acknowledge['uuid'] = status['uuid'] 365 | body = dict( 366 | type='ACKNOWLEDGE', 367 | framework_id=dict( 368 | value=framework_id, 369 | ), 370 | acknowledge=acknowledge, 371 | ) 372 | self._send(body) 373 | 374 | def acknowledgeOperationStatusUpdate(self, status): 375 | if self.connected and 'uuid' in status and 'operation_id' in status: 376 | framework_id = self.framework_id 377 | assert framework_id 378 | 379 | body = dict( 380 | type='ACKNOWLEDGE_OPERATION_STATUS', 381 | framework_id=dict( 382 | value=framework_id, 383 | ), 384 | acknowledge_operation_status=status, 385 | ) 386 | self._send(body) 387 | 388 | def reconcileTasks(self, tasks): 389 | if not self.connected: 390 | return 391 | 392 | framework_id = self.framework_id 393 | assert framework_id 394 | body = dict( 395 | type='RECONCILE', 396 | framework_id=dict( 397 | value=framework_id, 398 | ), 399 | reconcile=dict( 400 | tasks=[dict(task_id=task['task_id']) for task in tasks], 401 | ), 402 | ) 403 | self._send(body) 404 | 405 | def reconcileOperations(self, operations_): 406 | if not self.connected: 407 | return 408 | 409 | framework_id = self.framework_id 410 | assert framework_id 411 | operations = [] 412 | for op_ in operations_: 413 | op = dict( 414 | operation_id=op_['operation_id'] 415 | ) 416 | if 'agent_id' in op_: 417 | op['agent_id'] = op_['agent_id'] 418 | 419 | if 'resource_provider_id' in op_: 420 | op['resource_provider_id'] = op_['resource_provider_id'] 421 | 422 | operations.append(op) 423 | 424 | body = dict( 425 | type='RECONCILE_OPERATIONS', 426 | framework_id=dict( 427 | value=framework_id, 428 | ), 429 | reconcile_operations=dict( 430 | operations=operations, 431 | ), 432 | ) 433 | self._send(body) 434 | 435 | def sendFrameworkMessage(self, executor_id, agent_id, data): 436 | if not self.connected: 437 | return 438 | 439 | framework_id = self.framework_id 440 | assert framework_id 441 | message = dict( 442 | agent_id=agent_id, 443 | executor_id=executor_id, 444 | data=data, 445 | ) 446 | 447 | body = dict( 448 | type='MESSAGE', 449 | framework_id=dict( 450 | value=framework_id, 451 | ), 452 | message=message, 453 | ) 454 | self._send(body) 455 | 456 | def requestResources(self, requests): 457 | if not self.connected: 458 | return 459 | 460 | framework_id = self.framework_id 461 | assert framework_id 462 | body = dict( 463 | type='REQUEST', 464 | framework_id=dict( 465 | value=framework_id, 466 | ), 467 | request=dict( 468 | requests=requests, 469 | ), 470 | ) 471 | self._send(body) 472 | 473 | def onNewMasterDetectedMessage(self, data): 474 | master = None 475 | try: 476 | if isinstance(data, six.binary_type): 477 | data = data.decode('utf-8') 478 | 479 | parsed = json.loads(data) 480 | if parsed and "address" in parsed: 481 | ip = parsed["address"].get("ip") 482 | port = parsed["address"].get("port") 483 | if ip and port: 484 | master = "%s:%s" % (ip, port) 485 | except Exception: 486 | logger.exception("No JSON content, probably connecting " 487 | "to older Mesos version.") 488 | 489 | if master: 490 | self.change_master(master) 491 | 492 | def onNoMasterDetectedMessage(self): 493 | self.change_master(None) 494 | 495 | def gen_request(self): 496 | request = dict( 497 | type='SUBSCRIBE', 498 | subscribe=dict( 499 | framework_info=self.framework 500 | ), 501 | ) 502 | if 'id' in self._framework: 503 | request['framework_id'] = self._framework['id'] 504 | 505 | data = json.dumps(request) 506 | _authorization = '' 507 | if self._basic_credential is not None: 508 | _authorization = 'Authorization: %s\r\n' % ( 509 | self._basic_credential, 510 | ) 511 | 512 | request = ('POST /api/v1/scheduler HTTP/1.1\r\nHost: %s\r\n' 513 | 'Content-Type: application/json\r\n' 514 | 'Accept: application/json\r\n%s' 515 | 'Connection: close\r\nContent-Length: %s\r\n\r\n%s') % ( 516 | self.master, _authorization, len(data), data 517 | ) 518 | return request.encode('utf-8') 519 | 520 | def _close(self): 521 | if self._conn is not None: 522 | self._conn.close() 523 | self._conn = None 524 | 525 | def on_close(self): 526 | self._close() 527 | 528 | self.sched.disconnected(self) 529 | 530 | def on_subscribed(self, info): 531 | reregistered = (self.framework_id is not None) 532 | self.framework_id = info['framework_id']['value'] 533 | hostname, port = self.master.split(':', 2) 534 | port = int(port) 535 | master_info = dict( 536 | hostname=hostname, 537 | port=port, 538 | ) 539 | if self.version: 540 | master_info['version'] = self.version 541 | elif 'master_info' in info and 'version' in info['master_info']: 542 | master_info['version'] = info['master_info']['version'] 543 | 544 | if reregistered: 545 | self.sched.reregistered(self, self._dict_cls(master_info)) 546 | else: 547 | framework_id = dict( 548 | value=self.framework_id 549 | ) 550 | self.sched.registered( 551 | self, self._dict_cls(framework_id), 552 | self._dict_cls(master_info) 553 | ) 554 | 555 | def on_offers(self, event): 556 | offers = event.get('offers', []) 557 | if offers: 558 | self.sched.resourceOffers( 559 | self, [self._dict_cls(offer) for offer in offers] 560 | ) 561 | 562 | version = self.version and tuple( 563 | int(n) for n in self.version.split('.') 564 | ) 565 | 566 | if not (version and version >= (1, 0, 0)): 567 | self.on_inverse_offers(event) 568 | 569 | def on_inverse_offers(self, event): 570 | inverse_offers = event.get('inverse_offers', []) 571 | if inverse_offers: 572 | self.sched.inverseOffers( 573 | self, [self._dict_cls(offer) for offer in inverse_offers] 574 | ) 575 | 576 | def on_rescind(self, event): 577 | offer_id = event['offer_id'] 578 | self.sched.offerRescinded(self, self._dict_cls(offer_id)) 579 | 580 | def on_rescind_inverse_offer(self, event): 581 | inverse_offer_id = event['inverse_offer_id'] 582 | self.sched.inverseOfferRescinded( 583 | self, self._dict_cls(inverse_offer_id) 584 | ) 585 | 586 | def on_update(self, event): 587 | status = event['status'] 588 | self.sched.statusUpdate(self, self._dict_cls(status)) 589 | if self.implicit_acknowledgements: 590 | self.acknowledgeStatusUpdate(status) 591 | 592 | def on_update_operation_status(self, event): 593 | status = event['status'] 594 | self.sched.operationStatusUpdate(self, self._dict_cls(status)) 595 | if self.implicit_acknowledgements: 596 | self.acknowledgeOperationStatusUpdate(status) 597 | 598 | def on_message(self, message): 599 | executor_id = message['executor_id'] 600 | agent_id = message['agent_id'] 601 | data = message['data'] 602 | self.sched.frameworkMessage( 603 | self, self._dict_cls(executor_id), self._dict_cls(agent_id), data 604 | ) 605 | 606 | def on_failure(self, failure): 607 | agent_id = failure['agent_id'] 608 | if 'executor_id' not in failure: 609 | self.sched.slaveLost(self, self._dict_cls(agent_id)) 610 | else: 611 | self.sched.executorLost( 612 | self, self._dict_cls(failure['executor_id']), 613 | self._dict_cls(agent_id), failure['status'] 614 | ) 615 | 616 | def on_error(self, event): 617 | message = event['message'] 618 | self.sched.error(self, message) 619 | 620 | def on_heartbeat(self): 621 | self.sched.processHeartBeat(self) 622 | 623 | def on_event(self, event): 624 | if 'type' in event: 625 | _type = event['type'].lower() 626 | 627 | if _type == 'heartbeat': 628 | self.on_heartbeat() 629 | return 630 | 631 | if _type not in event: 632 | logger.error( 633 | 'Missing `%s` in event %s' % 634 | (_type, event)) 635 | return 636 | 637 | event = event[_type] 638 | func_name = 'on_%s' % (_type,) 639 | func = getattr(self, func_name, None) 640 | if func is not None: 641 | func(event) 642 | else: 643 | logger.error('Unknown type:%s, event:%s' % (_type, event)) 644 | else: 645 | logger.error('Unknown event:%s' % (event,)) 646 | -------------------------------------------------------------------------------- /pymesos/subprocess/__init__.py: -------------------------------------------------------------------------------- 1 | from subprocess import CalledProcessError 2 | from .popen import Popen, PIPE, STDOUT 3 | 4 | __all__ = ['Popen', 'PIPE', 'STDOUT', 'call', 'check_call', 5 | 'check_output', 'CalledProcessError'] 6 | 7 | 8 | def call(*popenargs, **kwargs): 9 | """Run command with arguments. Wait for command to complete, then 10 | return the returncode attribute. 11 | 12 | The arguments are the same as for the Popen constructor. Example: 13 | 14 | retcode = call(["ls", "-l"]) 15 | """ 16 | return Popen(*popenargs, **kwargs).wait() 17 | 18 | 19 | def check_call(*popenargs, **kwargs): 20 | """Run command with arguments. Wait for command to complete. If 21 | the exit code was zero then return, otherwise raise 22 | CalledProcessError. The CalledProcessError object will have the 23 | return code in the returncode attribute. 24 | 25 | The arguments are the same as for the Popen constructor. Example: 26 | 27 | check_call(["ls", "-l"]) 28 | """ 29 | retcode = call(*popenargs, **kwargs) 30 | if retcode: 31 | cmd = kwargs.get("args") 32 | if cmd is None: 33 | cmd = popenargs[0] 34 | raise CalledProcessError(retcode, cmd) 35 | return 0 36 | 37 | 38 | def check_output(*popenargs, **kwargs): 39 | r"""Run command with arguments and return its output as a byte string. 40 | 41 | If the exit code was non-zero it raises a CalledProcessError. The 42 | CalledProcessError object will have the return code in the returncode 43 | attribute and output in the output attribute. 44 | 45 | The arguments are the same as for the Popen constructor. Example: 46 | 47 | >>> check_output(["ls", "-l", "/dev/null"]) 48 | 'crw-rw-rw- 1 root root 1, 3 Oct 18 2007 /dev/null\n' 49 | 50 | The stdout argument is not allowed as it is used internally. 51 | To capture standard error in the result, use stderr=STDOUT. 52 | 53 | >>> check_output(["/bin/sh", "-c", 54 | ... "ls -l non_existent_file ; exit 0"], 55 | ... stderr=STDOUT) 56 | 'ls: non_existent_file: No such file or directory\n' 57 | """ 58 | if 'stdout' in kwargs: 59 | raise ValueError('stdout argument not allowed, it will be overridden.') 60 | process = Popen(stdout=PIPE, *popenargs, **kwargs) 61 | output, unused_err = process.communicate() 62 | retcode = process.poll() 63 | if retcode: 64 | cmd = kwargs.get("args") 65 | if cmd is None: 66 | cmd = popenargs[0] 67 | raise CalledProcessError(retcode, cmd, output=output) 68 | return output 69 | -------------------------------------------------------------------------------- /pymesos/subprocess/executor.py: -------------------------------------------------------------------------------- 1 | import os 2 | import sys 3 | import time 4 | import errno 5 | import socket 6 | import logging 7 | import traceback 8 | import subprocess 9 | from threading import Condition 10 | from six.moves import cPickle as pickle 11 | from .. import Executor, MesosExecutorDriver, encode_data, decode_data 12 | from .scheduler import _TYPE_SIGNAL 13 | 14 | logger = logging.getLogger(__name__) 15 | 16 | 17 | class ProcExecutor(Executor): 18 | 19 | def __init__(self): 20 | self.procs = {} 21 | self.pid_to_proc = {} 22 | self.cond = Condition() 23 | 24 | def registered(self, driver, executor_info, framework_info, agent_info): 25 | self.agent_id = agent_info['id'] 26 | 27 | def reregistered(self, driver, agent_info): 28 | self.agent_id = agent_info['id'] 29 | 30 | def abort(self): 31 | Executor.abort(self) 32 | self.cond.notify() 33 | 34 | def reply_status(self, driver, proc_id, state, message='', data=tuple()): 35 | update = dict( 36 | task_id=dict(value=str(proc_id)), 37 | agent_id=self.agent_id, 38 | timestamp=time.time(), 39 | state=state, 40 | ) 41 | 42 | if message: 43 | update['message'] = message 44 | 45 | if data: 46 | update['data'] = encode_data(pickle.dumps(data)) 47 | 48 | driver.sendStatusUpdate(update) 49 | 50 | def launchTask(self, driver, task): 51 | logger.info('Launch task') 52 | proc_id = int(task['task_id']['value']) 53 | self.reply_status(driver, proc_id, 'TASK_RUNNING') 54 | params = pickle.loads(decode_data(task['data'])) 55 | a = params['a'] 56 | kw = params['kw'] 57 | handlers = params['handlers'] 58 | hostname = params['hostname'] 59 | 60 | for i, key in enumerate(['stdin', 'stdout', 'stderr']): 61 | kw[key] = s = socket.socket() 62 | logger.info('Connect %s:%s for %s' % (hostname, handlers[i], key)) 63 | s.connect((hostname, handlers[i])) 64 | 65 | kw.pop('close_fds', None) 66 | 67 | try: 68 | p = subprocess.Popen(*a, close_fds=True, **kw) 69 | except Exception: 70 | exc_type, exc_value, tb = sys.exc_info() 71 | # Save the traceback and attach it to the exception object 72 | exc_lines = traceback.format_exception(exc_type, 73 | exc_value, 74 | tb) 75 | exc_value.child_traceback = ''.join(exc_lines) 76 | self.reply_status(driver, proc_id, 'TASK_FAILED', 77 | data=(None, exc_value)) 78 | logger.exception('Exec failed') 79 | return 80 | finally: 81 | kw['stdin'].close() 82 | kw['stdout'].close() 83 | kw['stderr'].close() 84 | 85 | with self.cond: 86 | self.procs[proc_id] = p 87 | self.pid_to_proc[p.pid] = proc_id 88 | self.cond.notify() 89 | 90 | def killTask(self, driver, task_id): 91 | logger.info('Kill task') 92 | with self.cond: 93 | proc_id = int(task_id['value']) 94 | if proc_id in self.procs: 95 | self.procs[proc_id].kill() 96 | 97 | def shutdown(self, driver): 98 | logger.info('Executor shutdown') 99 | with self.cond: 100 | for proc in list(self.procs.values()): 101 | proc.kill() 102 | 103 | def disconnected(self, driver): 104 | with self.cond: 105 | if driver.aborted: 106 | self.cond.notify() 107 | 108 | def run(self, driver): 109 | driver.start() 110 | while not driver.aborted: 111 | try: 112 | logger.debug('start waiting children...') 113 | pid, state = os.waitpid(-1, 0) 114 | logger.debug('stop waiting children...') 115 | 116 | with self.cond: 117 | if pid in self.pid_to_proc: 118 | proc_id = self.pid_to_proc.pop(pid) 119 | proc = self.procs.pop(proc_id) 120 | high = state >> 8 121 | low = state & 0x7F 122 | returncode = -low or high 123 | success = not low 124 | logger.info('Proc[%s:%s] terminated. success=%s, ' 125 | 'returncode=%s', proc_id, pid, success, 126 | returncode) 127 | if success: 128 | self.reply_status(driver, proc_id, 129 | 'TASK_FINISHED', 130 | data=(returncode, None)) 131 | else: 132 | self.reply_status(driver, proc_id, 133 | 'TASK_KILLED', 134 | data=(returncode, None)) 135 | 136 | except OSError as e: 137 | if e.errno != errno.ECHILD: 138 | raise 139 | 140 | with self.cond: 141 | while not driver.aborted and not self.procs: 142 | logger.debug('start waiting procs...') 143 | self.cond.wait() 144 | logger.debug('stop waiting procs...') 145 | 146 | with self.cond: 147 | for proc in list(self.procs.values()): 148 | proc.kill() 149 | self.reply_status(driver, proc_id, 150 | 'TASK_KILLED') 151 | self.pid_to_proc.clear() 152 | self.procs.clear() 153 | 154 | driver.join() 155 | 156 | def frameworkMessage(self, driver, msg): 157 | pid, type, data = pickle.loads(decode_data(msg)) 158 | logger.info('Recv framework message pid:%s, type:%s, data:%s', 159 | pid, type, data) 160 | 161 | with self.cond: 162 | if pid not in self.procs: 163 | logger.error('Cannot find pid:%s to send message', pid) 164 | return 165 | 166 | p = self.procs[pid] 167 | if type == _TYPE_SIGNAL: 168 | sig = int(data) 169 | p.send_signal(sig) 170 | 171 | 172 | if __name__ == '__main__': 173 | log_format = '%(asctime)-15s [%(levelname)s] [%(name)-9s] %(message)s' 174 | logging.basicConfig(format=log_format, level=logging.DEBUG) 175 | executor = ProcExecutor() 176 | driver = MesosExecutorDriver(executor) 177 | executor.run(driver) 178 | -------------------------------------------------------------------------------- /pymesos/subprocess/popen.py: -------------------------------------------------------------------------------- 1 | import os 2 | import sys 3 | import time 4 | import atexit 5 | import select 6 | import socket 7 | import signal 8 | import logging 9 | from six import string_types 10 | from threading import RLock, Thread, Condition 11 | from subprocess import PIPE, STDOUT 12 | from .scheduler import ProcScheduler, CONFIG, _TYPE_SIGNAL 13 | 14 | logger = logging.getLogger(__name__) 15 | PIPE_BUF = getattr(select, 'PIPE_BUF', 4096) 16 | 17 | 18 | class Redirector(object): 19 | 20 | def __init__(self): 21 | rfd, wfd = os.pipe() 22 | self._listeners = {} 23 | self._readers = {} 24 | self._writers = {} 25 | self._proc_fds = {} 26 | self._proc_callback = {} 27 | self._lock = RLock() 28 | self._wakeup_fd = wfd 29 | self._aborted = False 30 | self._io_thread = Thread( 31 | target=Redirector._loop, args=(self, rfd), 32 | name='Redirector' 33 | ) 34 | self._io_thread.daemon = True 35 | self._io_thread.start() 36 | 37 | def _clear(self, fd): 38 | r = self._listeners.pop(fd, None) \ 39 | or self._readers.pop(fd, None) \ 40 | or self._writers.pop(fd, None) 41 | if not r: 42 | fd.close() 43 | return 44 | 45 | f, pid = r[:2] 46 | self._proc_fds[pid].remove(fd) 47 | if not self._proc_fds[pid]: 48 | del self._proc_fds[pid] 49 | callback = self._proc_callback.pop(pid, None) 50 | if callback: 51 | callback() 52 | 53 | fd.close() 54 | f.close() 55 | 56 | def _loop(self, rfd): 57 | MINIMAL_INTERVAL = 0.1 58 | BUFFER_SIZE = PIPE_BUF 59 | while True: 60 | more_to_read = False 61 | with self._lock: 62 | if self._aborted: 63 | break 64 | to_read = [rfd] + list(self._listeners.keys()) 65 | to_read += list(self._writers.keys()) + \ 66 | list(self._readers.keys()) 67 | to_write = list(self._readers.keys()) 68 | 69 | readable, writeable, _ = select.select(to_read, to_write, []) 70 | with self._lock: 71 | for fd in readable: 72 | if fd in self._listeners: 73 | _fd, _ = fd.accept() 74 | logger.info('[%s]Accept conn:%s' % (fd, _fd)) 75 | f, pid, readonly = self._listeners.pop(fd) 76 | self._proc_fds[pid].remove(fd) 77 | self._proc_fds[pid].add(_fd) 78 | fd.close() 79 | if readonly: 80 | self._readers[_fd] = (f, pid) 81 | else: 82 | self._writers[_fd] = (f, pid) 83 | 84 | elif fd in self._writers: 85 | f, pid = self._writers[fd] 86 | ret = select.select([], [f], [], 0) 87 | if not ret[1]: 88 | continue 89 | 90 | buf = fd.recv(BUFFER_SIZE) 91 | if buf: 92 | f.write(buf) 93 | more_to_read = bool( 94 | select.select([fd], [], [], 0)[0]) 95 | else: 96 | logger.info('Closing writer %s', fd) 97 | self._clear(fd) 98 | 99 | elif fd in self._readers: 100 | logger.info('Closing reader %s', fd) 101 | self._clear(fd) 102 | 103 | elif fd == rfd: 104 | os.read(rfd, BUFFER_SIZE) 105 | 106 | else: 107 | fd.close() 108 | 109 | for fd in writeable: 110 | if fd in self._readers: 111 | f, pid = self._readers[fd] 112 | ret = select.select([f], [], [], 0) 113 | if not ret[0]: 114 | continue 115 | 116 | buf = f.read(BUFFER_SIZE) 117 | if buf: 118 | try: 119 | fd.sendall(buf) 120 | more_to_read = bool( 121 | select.select([f], [], [])[0]) 122 | except socket.error: 123 | logger.info('Closing reader %s', fd) 124 | self._clear(fd) 125 | else: 126 | logger.info('Closing reader %s', fd) 127 | self._clear(fd) 128 | 129 | else: 130 | fd.close() 131 | 132 | if not more_to_read: 133 | with self._lock: 134 | to_read = [rfd] + list(self._listeners.keys()) 135 | 136 | select.select(to_read, [], [], MINIMAL_INTERVAL) 137 | 138 | for pid in list(self._proc_fds.keys()): 139 | for fd in list(self._proc_fds.get(pid, [])): 140 | self._clear(fd) 141 | 142 | os.close(rfd) 143 | 144 | def _wakeup(self): 145 | os.write(self._wakeup_fd, '\0') 146 | 147 | def _register(self, pid, f, readonly=False): 148 | lfd = socket.socket() 149 | lfd.bind(('0.0.0.0', 0)) 150 | _, port = lfd.getsockname() 151 | lfd.listen(1) 152 | logger.info('Listen %s for %s' % (port, f)) 153 | with self._lock: 154 | self._listeners[lfd] = (f, pid, readonly) 155 | self._proc_fds[pid].add(lfd) 156 | 157 | self._wakeup() 158 | 159 | return port 160 | 161 | def unregister(self, pid): 162 | with self._lock: 163 | logger.info('Unregister %s', pid) 164 | for fd in list(self._proc_fds.get(pid, [])): 165 | self._clear(fd) 166 | 167 | self._wakeup() 168 | 169 | def register(self, pid, stdin, stdout, stderr, callback=None): 170 | with self._lock: 171 | self._proc_fds[pid] = set() 172 | self._proc_callback[pid] = callback 173 | 174 | return (self._register(pid, stdin, readonly=True), 175 | self._register(pid, stdout), 176 | self._register(pid, stderr)) 177 | 178 | def stop(self): 179 | with self._lock: 180 | self._aborted = True 181 | os.close(self._wakeup_fd) 182 | self._io_thread.join() 183 | 184 | 185 | _STARTING, _RUNNING, _STOPPED = list(range(3)) 186 | _STARTING_TIMEOUT = 5 * 60 187 | UNKNOWN_ERROR = -255 188 | 189 | 190 | class Popen(object): 191 | _next_id = 0 192 | _scheduler = None 193 | _redirector = None 194 | _lock = RLock() 195 | 196 | def __init__(self, args, bufsize=0, executable=None, 197 | stdin=None, stdout=None, stderr=None, 198 | preexec_fn=None, close_fds=None, shell=False, 199 | cwd=None, env=None, universal_newlines=False, 200 | startupinfo=None, creationflags=0, 201 | cpus=None, mem=None, gpus=None): 202 | 203 | kw = dict(list(locals().items())) 204 | a = (args,) 205 | 206 | kw.pop('self') 207 | kw.pop('args') 208 | close_fds = kw.pop('close_fds') 209 | if close_fds is False: 210 | logger.warning('Can not support `close_fds=False`, ' 211 | 'no fds will be inherrited.') 212 | 213 | stdin = kw.pop('stdin', None) 214 | stdout = kw.pop('stdout', None) 215 | stderr = kw.pop('stderr', None) 216 | cpus = kw.pop('cpus', None) 217 | mem = kw.pop('mem', None) 218 | gpus = kw.pop('gpus', None) 219 | 220 | kw['cwd'] = kw.get('cwd') or os.getcwd() 221 | kw['env'] = kw.get('env') or dict(list(os.environ.items())) 222 | 223 | self.id = self._new_id() 224 | self.cpus = float(cpus or CONFIG.get('default_cpus', 1.0)) 225 | self.mem = float(mem or CONFIG.get('default_mem', 1024.0)) 226 | self.gpus = int(gpus or CONFIG.get('default_gpus', 0)) 227 | self.pid = None 228 | self.returncode = None 229 | self._returncode = None 230 | self._a = a 231 | self._kw = kw 232 | self._exc = None 233 | self._state = _STARTING 234 | self._io_waiting = True 235 | self._cond = Condition() 236 | 237 | self._prepare_handlers(stdin, stdout, stderr) 238 | self._submit() 239 | with self._cond: 240 | deadline = time.time() + _STARTING_TIMEOUT 241 | while True: 242 | if self._state != _STARTING: 243 | break 244 | 245 | delta = deadline - time.time() 246 | if deadline <= 0: 247 | raise RuntimeError('Too long to start!') 248 | 249 | self._cond.wait(delta) 250 | 251 | if self._exc: 252 | raise self._exc 253 | 254 | @classmethod 255 | def _new_id(cls): 256 | cls._next_id += 1 257 | return cls._next_id 258 | 259 | def _submit(self): 260 | cls = self.__class__ 261 | with cls._lock: 262 | if not cls._scheduler: 263 | cls._scheduler = ProcScheduler() 264 | cls._scheduler.start() 265 | atexit.register(cls._scheduler.stop) 266 | 267 | cls._scheduler.submit(self) 268 | 269 | def _prepare_handlers(self, stdin, stdout, stderr): 270 | def _dup_file(f, *a, **kw): 271 | fd = f if isinstance(f, int) else f.fileno() 272 | return os.fdopen(os.dup(fd), *a, **kw) 273 | 274 | cls = self.__class__ 275 | if not cls._redirector: 276 | cls._redirector = Redirector() 277 | atexit.register(cls._redirector.stop) 278 | 279 | if stdin == PIPE: 280 | r, w = os.pipe() 281 | _in = os.fdopen(r, 'rb', 0) 282 | self.stdin = os.fdopen(w, 'wb', 0) 283 | else: 284 | self.stdin = None 285 | if stdin is None: 286 | _in = _dup_file(sys.stdin, 'rb', 0) 287 | else: 288 | _in = _dup_file(stdin, 'rb', 0) 289 | 290 | if stdout == PIPE: 291 | r, w = os.pipe() 292 | _out = os.fdopen(w, 'wb', 0) 293 | self.stdout = os.fdopen(r, 'rb', 0) 294 | else: 295 | self.stdout = None 296 | if stdout is None: 297 | _out = _dup_file(sys.stdout, 'wb', 0) 298 | else: 299 | _out = _dup_file(stdout, 'wb', 0) 300 | 301 | if stderr == PIPE: 302 | r, w = os.pipe() 303 | _err = os.fdopen(w, 'wb', 0) 304 | self.stderr = os.fdopen(r, 'rb', 0) 305 | else: 306 | self.stderr = None 307 | if stderr is None: 308 | _err = _dup_file(sys.stderr, 'wb', 0) 309 | elif stderr == STDOUT: 310 | _err = _dup_file(_out, 'wb', 0) 311 | else: 312 | _err = _dup_file(sys.stderr, 'wb', 0) 313 | 314 | self._handlers = cls._redirector.register( 315 | self.id, _in, _out, _err, self._io_complete) 316 | 317 | def _io_complete(self): 318 | with self._cond: 319 | self._io_waiting = False 320 | self._cond.notify() 321 | 322 | def _kill(self): 323 | cls = self.__class__ 324 | cls._redirector.unregister(self.id) 325 | 326 | def __repr__(self): 327 | args = self._a[0] 328 | if isinstance(args, string_types): 329 | cmd = args 330 | else: 331 | cmd = ' '.join(args) 332 | 333 | return '%s: %s' % (self.__class__.__name__, cmd) 334 | 335 | @property 336 | def params(self): 337 | return dict( 338 | a=self._a, 339 | kw=self._kw, 340 | cpus=self.cpus, 341 | mem=self.mem, 342 | gpus=self.gpus, 343 | handlers=self._handlers, 344 | hostname=socket.gethostname(), 345 | ) 346 | 347 | def _started(self): 348 | logger.info('Started') 349 | with self._cond: 350 | self._state = _RUNNING 351 | self._cond.notify() 352 | 353 | def _finished(self, success, message, data): 354 | logger.info('Sucess:%s message:%s', success, message) 355 | if success: 356 | self._returncode, self._exc = data 357 | else: 358 | self._returncode, self._exc = data or ( 359 | UNKNOWN_ERROR, None) 360 | self._kill() 361 | 362 | with self._cond: 363 | self._state = _STOPPED 364 | self._cond.notify() 365 | 366 | def poll(self): 367 | with self._cond: 368 | if self._state == _STOPPED and not self._io_waiting: 369 | if self.stdin and not self.stdin.closed: 370 | self.stdin.close() 371 | self.stdin = None 372 | 373 | if self.stdout and not self.stdout.closed: 374 | self.stdout.close() 375 | self.stdout = None 376 | 377 | if self.stderr and not self.stderr.closed: 378 | self.stderr.close() 379 | self.stderr = None 380 | 381 | self.returncode = self._returncode 382 | 383 | return self.returncode 384 | 385 | def wait(self): 386 | with self._cond: 387 | while self.poll() is None: 388 | self._cond.wait() 389 | 390 | return self.returncode 391 | 392 | def communicate(self, input=None): 393 | BUFFER_SIZE = PIPE_BUF 394 | buf = input and input[:] 395 | out = None 396 | err = None 397 | to_write = [_f for _f in [self.stdin] if _f] 398 | to_read = [_f for _f in [self.stdout, self.stderr] if _f] 399 | 400 | while True: 401 | can_wait = True 402 | readable, writeable, _ = select.select(to_read, to_write, [], 0) 403 | if writeable: 404 | if buf: 405 | can_wait = False 406 | size = os.write(self.stdin.fileno(), buf[:BUFFER_SIZE]) 407 | buf = buf[size:] 408 | else: 409 | if self.stdin and not self.stdin.closed: 410 | self.stdin.close() 411 | 412 | to_write = [] 413 | 414 | if readable: 415 | can_wait = False 416 | if self.stdout in readable: 417 | if out is None: 418 | out = self.stdout.read(BUFFER_SIZE) 419 | else: 420 | out += self.stdout.read(BUFFER_SIZE) 421 | 422 | if self.stderr in readable: 423 | if err is None: 424 | err = self.stderr.read(BUFFER_SIZE) 425 | else: 426 | err += self.stderr.read(BUFFER_SIZE) 427 | 428 | with self._cond: 429 | if self.poll() is None: 430 | if can_wait: 431 | self._cond.wait(0.1) 432 | 433 | else: 434 | return (out, err) 435 | 436 | def send_signal(self, signal): 437 | cls = self.__class__ 438 | cls._scheduler.send_data(self.id, _TYPE_SIGNAL, signal) 439 | 440 | def terminate(self): 441 | self.send_signal(signal.SIGTERM) 442 | 443 | def kill(self): 444 | self.send_signal(signal.SIGKILL) 445 | 446 | def cancel(self): 447 | cls = self.__class__ 448 | cls._scheduler.cancel(self) 449 | self._kill() 450 | self.returncode = UNKNOWN_ERROR 451 | -------------------------------------------------------------------------------- /pymesos/subprocess/scheduler.py: -------------------------------------------------------------------------------- 1 | import os 2 | import sys 3 | import random 4 | import socket 5 | import getpass 6 | import logging 7 | from threading import RLock 8 | from six.moves import cPickle as pickle 9 | from .. import Scheduler, MesosSchedulerDriver, encode_data, decode_data 10 | 11 | logger = logging.getLogger(__name__) 12 | CONFIG = {} 13 | FOREVER = 0xFFFFFFFF 14 | _TYPE_SIGNAL, = list(range(1)) 15 | MIN_CPUS = 0.01 16 | MIN_MEMORY = 32 17 | 18 | 19 | class ProcScheduler(Scheduler): 20 | 21 | def __init__(self): 22 | self.framework_id = None 23 | self.framework = self._init_framework() 24 | self.executor = None 25 | self.master = str(CONFIG.get('master', os.environ['MESOS_MASTER'])) 26 | self.principal = str( 27 | CONFIG.get('principal', os.environ.get('DEFAULT_PRINCIPAL')) 28 | ) 29 | self.secret = str( 30 | CONFIG.get('secret', os.environ.get('DEFAULT_SECRET')) 31 | ) 32 | self.driver = MesosSchedulerDriver( 33 | self, self.framework, self.master, principal=self.principal, 34 | secret=self.secret 35 | ) 36 | self.procs_pending = {} 37 | self.procs_launched = {} 38 | self.agent_to_proc = {} 39 | self._lock = RLock() 40 | 41 | def _init_framework(self): 42 | framework = dict( 43 | user=getpass.getuser(), 44 | name=repr(self), 45 | hostname=socket.gethostname(), 46 | ) 47 | return framework 48 | 49 | def _init_executor(self): 50 | executor = dict( 51 | executor_id=dict(value='default'), 52 | framework_id=self.framework_id, 53 | command=dict( 54 | value='%s -m %s.executor' % ( 55 | sys.executable, __package__ 56 | ) 57 | ), 58 | resources=[ 59 | dict( 60 | name='mem', 61 | type='SCALAR', 62 | scalar=dict(value=MIN_MEMORY), 63 | ), 64 | dict( 65 | name='cpus', 66 | type='SCALAR', 67 | scalar=dict(value=MIN_CPUS) 68 | ), 69 | ], 70 | ) 71 | 72 | if 'PYTHONPATH' in os.environ: 73 | executor['command.environment'] = dict( 74 | variables=[ 75 | dict( 76 | name='PYTHONPATH', 77 | value=os.environ['PYTHONPATH'], 78 | ), 79 | ] 80 | ) 81 | 82 | return executor 83 | 84 | def _init_task(self, proc, offer): 85 | resources = [ 86 | dict( 87 | name='cpus', 88 | type='SCALAR', 89 | scalar=dict(value=proc.cpus), 90 | ), 91 | dict( 92 | name='mem', 93 | type='SCALAR', 94 | scalar=dict(value=proc.mem), 95 | ) 96 | ] 97 | 98 | if proc.gpus > 0: 99 | resources.append( 100 | dict( 101 | name='gpus', 102 | type='SCALAR', 103 | scalar=dict(value=proc.gpus), 104 | ) 105 | ) 106 | 107 | task = dict( 108 | task_id=dict(value=str(proc.id)), 109 | name=repr(proc), 110 | executor=self.executor, 111 | agent_id=offer['agent_id'], 112 | data=encode_data(pickle.dumps(proc.params)), 113 | resources=resources, 114 | ) 115 | 116 | return task 117 | 118 | def _filters(self, seconds): 119 | f = dict(refuse_seconds=seconds) 120 | return f 121 | 122 | def __repr__(self): 123 | return "%s[%s]: %s" % ( 124 | self.__class__.__name__, 125 | os.getpid(), ' '.join(sys.argv)) 126 | 127 | def registered(self, driver, framework_id, master_info): 128 | with self._lock: 129 | logger.info('Framework registered with id=%s, master=%s' % ( 130 | framework_id, master_info)) 131 | self.framework_id = framework_id 132 | self.executor = self._init_executor() 133 | 134 | def resourceOffers(self, driver, offers): 135 | def get_resources(offer): 136 | cpus, mem, gpus = 0.0, 0.0, 0 137 | for r in offer['resources']: 138 | if r['name'] == 'cpus': 139 | cpus = float(r['scalar']['value']) 140 | elif r['name'] == 'mem': 141 | mem = float(r['scalar']['value']) 142 | elif r['name'] == 'gpus': 143 | gpus = int(r['scalar']['value']) 144 | 145 | return cpus, mem, gpus 146 | 147 | with self._lock: 148 | random.shuffle(offers) 149 | for offer in offers: 150 | if not self.procs_pending: 151 | logger.debug('Reject offers forever for no pending procs, ' 152 | 'offers=%s' % (offers, )) 153 | driver.declineOffer( 154 | offer['id'], self._filters(FOREVER)) 155 | continue 156 | 157 | cpus, mem, gpus = get_resources(offer) 158 | tasks = [] 159 | for proc in list(self.procs_pending.values()): 160 | if (cpus >= proc.cpus + MIN_CPUS 161 | and mem >= proc.mem + MIN_MEMORY 162 | and gpus >= proc.gpus): 163 | tasks.append(self._init_task(proc, offer)) 164 | del self.procs_pending[proc.id] 165 | self.procs_launched[proc.id] = proc 166 | cpus -= proc.cpus 167 | mem -= proc.mem 168 | gpus -= proc.gpus 169 | 170 | seconds = 5 + random.random() * 5 171 | if tasks: 172 | logger.info('Accept offer for procs, offer=%s, ' 173 | 'procs=%s, filter_time=%s' % ( 174 | offer, 175 | [int(t['task_id']['value']) 176 | for t in tasks], 177 | seconds)) 178 | driver.launchTasks( 179 | offer['id'], tasks, self._filters(seconds)) 180 | else: 181 | logger.info('Retry offer for procs later, offer=%s, ' 182 | 'filter_time=%s' % ( 183 | offer, seconds)) 184 | driver.declineOffer(offer['id'], self._filters(seconds)) 185 | 186 | def _call_finished(self, proc_id, success, message, data, agent_id=None): 187 | with self._lock: 188 | proc = self.procs_launched.pop(proc_id) 189 | if agent_id is not None: 190 | if agent_id in self.agent_to_proc: 191 | self.agent_to_proc[agent_id].remove(proc_id) 192 | else: 193 | for agent_id, procs in list(self.agent_to_proc.items()): 194 | if proc_id in procs: 195 | procs.remove(proc_id) 196 | 197 | proc._finished(success, message, data) 198 | 199 | def statusUpdate(self, driver, update): 200 | with self._lock: 201 | proc_id = int(update['task_id']['value']) 202 | state = update['state'] 203 | logger.info('Status update for proc, id=%s, state=%s' % ( 204 | proc_id, state)) 205 | agent_id = update['agent_id']['value'] 206 | if proc_id not in self.procs_launched: 207 | logger.warning( 208 | 'Unknown proc %s update for %s ignored', 209 | proc_id, state 210 | ) 211 | driver.reviveOffers() 212 | return 213 | 214 | if state == 'TASK_RUNNING': 215 | if agent_id in self.agent_to_proc: 216 | self.agent_to_proc[agent_id].add(proc_id) 217 | else: 218 | self.agent_to_proc[agent_id] = set([proc_id]) 219 | 220 | proc = self.procs_launched[proc_id] 221 | proc._started() 222 | 223 | elif state not in { 224 | 'TASK_STAGING', 'TASK_STARTING', 'TASK_RUNNING' 225 | }: 226 | success = (state == 'TASK_FINISHED') 227 | message = update.get('message') 228 | data = update.get('data') 229 | if data: 230 | data = pickle.loads(decode_data(data)) 231 | 232 | self._call_finished(proc_id, success, message, data, agent_id) 233 | driver.reviveOffers() 234 | 235 | def offerRescinded(self, driver, offer_id): 236 | with self._lock: 237 | if self.procs_pending: 238 | logger.info('Revive offers for pending procs') 239 | driver.reviveOffers() 240 | 241 | def executorLost(self, driver, executor_id, agent_id, status): 242 | agent_id = agent_id['value'] 243 | with self._lock: 244 | for proc_id in self.agent_to_proc.pop(agent_id, []): 245 | self._call_finished( 246 | proc_id, False, 'Executor lost', None, agent_id) 247 | 248 | def slaveLost(self, driver, agent_id): 249 | agent_id = agent_id['value'] 250 | with self._lock: 251 | for proc_id in self.agent_to_proc.pop(agent_id, []): 252 | self._call_finished( 253 | proc_id, False, 'Agent lost', None, agent_id) 254 | 255 | def error(self, driver, message): 256 | with self._lock: 257 | for proc in list(self.procs_pending.values()): 258 | self._call_finished(proc.id, False, message, None) 259 | 260 | for proc in list(self.procs_launched.values()): 261 | self._call_finished(proc.id, False, message, None) 262 | 263 | self.stop() 264 | 265 | def start(self): 266 | self.driver.start() 267 | 268 | def stop(self): 269 | assert not self.driver.aborted 270 | self.driver.stop() 271 | self.driver.join() 272 | 273 | def submit(self, proc): 274 | if self.driver.aborted: 275 | raise RuntimeError('driver already aborted') 276 | 277 | with self._lock: 278 | if proc.id not in self.procs_pending: 279 | logger.info('Try submit proc, id=%s', (proc.id,)) 280 | self.procs_pending[proc.id] = proc 281 | if len(self.procs_pending) == 1: 282 | logger.info('Revive offers for pending procs') 283 | self.driver.reviveOffers() 284 | else: 285 | raise ValueError('Proc with same id already submitted') 286 | 287 | def cancel(self, proc): 288 | if self.driver.aborted: 289 | raise RuntimeError('driver already aborted') 290 | 291 | with self._lock: 292 | if proc.id in self.procs_pending: 293 | del self.procs_pending[proc.id] 294 | elif proc.id in self.procs_launched: 295 | del self.procs_launched[proc.id] 296 | self.driver.killTask(dict(value=str(proc.id))) 297 | 298 | for agent_id, procs in list(self.agent_to_proc.items()): 299 | procs.pop(proc.id) 300 | if not procs: 301 | del self.agent_to_proc[agent_id] 302 | 303 | def send_data(self, pid, type, data): 304 | if self.driver.aborted: 305 | raise RuntimeError('driver already aborted') 306 | 307 | msg = encode_data(pickle.dumps((pid, type, data))) 308 | for agent_id, procs in list(self.agent_to_proc.items()): 309 | if pid in procs: 310 | self.driver.sendFrameworkMessage( 311 | self.executor['executor_id'], 312 | dict(value=agent_id), 313 | msg 314 | ) 315 | return 316 | 317 | raise RuntimeError('Cannot find agent for pid %s' % (pid,)) 318 | -------------------------------------------------------------------------------- /pymesos/utils.py: -------------------------------------------------------------------------------- 1 | from binascii import b2a_base64, a2b_base64 2 | 3 | POSTFIX = { 4 | 'ns': 1e-9, 5 | 'us': 1e-6, 6 | 'ms': 1e-3, 7 | 'secs': 1, 8 | 'mins': 60, 9 | 'hrs': 60 * 60, 10 | 'days': 24 * 60 * 60, 11 | 'weeks': 7 * 24 * 60 * 60 12 | } 13 | 14 | DAY = 86400 # POSIX day 15 | 16 | 17 | def parse_duration(s): 18 | s = s.strip() 19 | t = None 20 | unit = None 21 | for n, u in POSTFIX.items(): 22 | if s.endswith(n): 23 | try: 24 | t = float(s[:-len(n)]) 25 | except ValueError: 26 | continue 27 | 28 | unit = u 29 | break 30 | 31 | assert unit is not None, \ 32 | 'Unknown duration \'%s\'; supported units are %s' % ( 33 | s, ','.join('\'%s\'' % n for n in POSTFIX) 34 | ) 35 | 36 | return t * unit 37 | 38 | 39 | def encode_data(data): 40 | return b2a_base64(data).strip().decode('ascii') 41 | 42 | 43 | def decode_data(data): 44 | return a2b_base64(data) 45 | -------------------------------------------------------------------------------- /pymesos/zkpython.py: -------------------------------------------------------------------------------- 1 | import zookeeper 2 | import threading 3 | import logging 4 | 5 | logger = logging.getLogger(__name__) 6 | logger.setLevel(logging.WARNING) 7 | zookeeper.set_debug_level(zookeeper.LOG_LEVEL_WARN) 8 | 9 | 10 | # Mapping of connection state values to human strings. 11 | STATE_NAME_MAPPING = { 12 | zookeeper.ASSOCIATING_STATE: "associating", 13 | zookeeper.AUTH_FAILED_STATE: "auth-failed", 14 | zookeeper.CONNECTED_STATE: "connected", 15 | zookeeper.CONNECTING_STATE: "connecting", 16 | zookeeper.EXPIRED_SESSION_STATE: "expired", 17 | } 18 | 19 | # Mapping of event type to human string. 20 | TYPE_NAME_MAPPING = { 21 | zookeeper.NOTWATCHING_EVENT: "not-watching", 22 | zookeeper.SESSION_EVENT: "session", 23 | zookeeper.CREATED_EVENT: "created", 24 | zookeeper.DELETED_EVENT: "deleted", 25 | zookeeper.CHANGED_EVENT: "changed", 26 | zookeeper.CHILD_EVENT: "child", 27 | } 28 | 29 | 30 | class TimeoutException(zookeeper.ZooKeeperException): 31 | pass 32 | 33 | 34 | def logevent(h, typ, state, path): 35 | logger.debug( 36 | "event,handle:%d, type:%s, state:%s, path:%s", 37 | h, TYPE_NAME_MAPPING.get( 38 | typ, "unknown"), STATE_NAME_MAPPING.get( 39 | state, "unknown"), path) 40 | 41 | 42 | class ZKClient: 43 | 44 | def __init__(self, servers, timeout=10): 45 | self.timeout = timeout 46 | self.connected = False 47 | self.handle = -1 48 | self.servers = servers 49 | self.watchers = set() 50 | self._lock = threading.Lock() 51 | self.conn_cv = threading.Condition() 52 | 53 | def start(self): 54 | self.handle = zookeeper.init( 55 | self.servers, 56 | self.connection_watcher, 57 | self.timeout * 1000) 58 | self.conn_cv.acquire() 59 | self.conn_cv.wait(self.timeout) 60 | self.conn_cv.release() 61 | if not self.connected: 62 | raise TimeoutException 63 | 64 | def stop(self): 65 | return zookeeper.close(self.handle) 66 | 67 | def connection_watcher(self, h, typ, state, path): 68 | logevent(h, typ, state, path) 69 | if typ == zookeeper.SESSION_EVENT: 70 | if state == zookeeper.CONNECTED_STATE: 71 | self.handle = h 72 | with self._lock: 73 | self.connected = True 74 | watchers = list(self.watchers) 75 | for watcher in watchers: 76 | watcher.watch() 77 | 78 | self.conn_cv.acquire() 79 | self.conn_cv.notifyAll() 80 | self.conn_cv.release() 81 | 82 | def del_watcher(self, watcher): 83 | with self._lock: 84 | self.watchers.discard(watcher) 85 | 86 | def add_watcher(self, watcher): 87 | with self._lock: 88 | self.watchers.add(watcher) 89 | if self.connected: 90 | watcher.watch() 91 | 92 | 93 | class DataWatch: 94 | 95 | def __init__(self, client, path, func): 96 | self._client = client 97 | self._path = path 98 | self._func = func 99 | self._stopped = False 100 | client.add_watcher(self) 101 | 102 | def watcher(self, h, typ, state, path): 103 | logevent(h, typ, state, path) 104 | self.watch() 105 | 106 | def _do(self): 107 | data, stat = zookeeper.get( 108 | self._client.handle, self._path, self.watcher) 109 | return self._func(data, stat) 110 | 111 | def watch(self): 112 | if self._stopped: 113 | return 114 | try: 115 | result = self._do() 116 | if result is False: 117 | self._stopped = True 118 | except zookeeper.NoNodeException: 119 | raise 120 | except zookeeper.ZooKeeperException as e: 121 | logger.error("ZooKeeperException, type:%s, msg: %s", type(e), e) 122 | 123 | 124 | class ChildrenWatch(DataWatch): 125 | 126 | def _do(self): 127 | children = zookeeper.get_children( 128 | self._client.handle, self._path, self.watcher) 129 | return self._func(children) 130 | -------------------------------------------------------------------------------- /requirements.txt: -------------------------------------------------------------------------------- 1 | addict==2.2.1 2 | coverage==4.2 3 | funcsigs==1.0.2 4 | http-parser==0.9.0 5 | mock==2.0.0 6 | pbr==1.10.0 7 | py==1.10.0 8 | pytest==3.0.2 9 | pytest-cov==2.3.1 10 | pytest-mock==1.2 11 | pytest-randomly==1.1.0 12 | pytest-runner==2.9 13 | six==1.10.0 14 | zkpython==0.4.2 15 | kazoo==2.7.0 16 | -------------------------------------------------------------------------------- /setup.cfg: -------------------------------------------------------------------------------- 1 | [metadata] 2 | description-file = README.rst 3 | 4 | [aliases] 5 | test=pytest 6 | 7 | [coverage:run] 8 | omit = 9 | pymesos/subprocess/* 10 | pymesos/zkpython.py 11 | pymesos/detector.py 12 | -------------------------------------------------------------------------------- /setup.py: -------------------------------------------------------------------------------- 1 | import os 2 | import re 3 | import sys 4 | from setuptools import setup, find_packages 5 | 6 | needs_pytest = {'pytest', 'test', 'ptr'}.intersection(sys.argv) 7 | pytest_runner = ['pytest-runner'] if needs_pytest else [] 8 | 9 | 10 | def find_version(*paths): 11 | fname = os.path.join(*paths) 12 | with open(fname) as fhandler: 13 | version_file = fhandler.read() 14 | version_match = re.search(r"^__VERSION__ = ['\"]([^'\"]*)['\"]", 15 | version_file, re.M) 16 | 17 | if not version_match: 18 | raise RuntimeError("Unable to find version string in %s" % (fname, )) 19 | 20 | version = version_match.group(1) 21 | return version 22 | 23 | 24 | version = find_version('pymesos', '__init__.py') 25 | install_requires=[ 26 | 'six', 27 | 'http-parser', 28 | 'addict', 29 | ] 30 | PY3 = sys.version_info > (3, ) 31 | PYPY = getattr(sys, 'pypy_version_info', False) and True or False 32 | 33 | if (PY3 or PYPY): 34 | install_requires += ['kazoo'] 35 | else: 36 | install_requires += ['zkpython'] 37 | print(install_requires) 38 | setup( 39 | name='pymesos', 40 | version=version, 41 | description="A pure python implementation of Mesos scheduler and executor", 42 | packages=find_packages(), 43 | platforms=['POSIX'], 44 | classifiers=[ 45 | 'Intended Audience :: Developers', 46 | 'License :: OSI Approved :: BSD License', 47 | 'Operating System :: POSIX', 48 | 'Programming Language :: Python', 49 | ], 50 | keywords='mesos', 51 | author="Zhongbo Tian", 52 | author_email="tianzhongbo@douban.com", 53 | url="https://github.com/douban/pymesos", 54 | download_url=('https://github.com/douban/pymesos/archive/%s.tar.gz' % 55 | version), 56 | install_requires=install_requires, 57 | setup_requires=pytest_runner, 58 | tests_require=['pytest-cov', 'pytest-randomly', 'pytest-mock', 'pytest'], 59 | ) 60 | -------------------------------------------------------------------------------- /tests/test_executor.py: -------------------------------------------------------------------------------- 1 | import json 2 | import uuid 3 | import random 4 | import string 5 | from http_parser.http import HttpParser 6 | from pymesos import MesosExecutorDriver, encode_data 7 | 8 | 9 | def test_gen_request(mocker): 10 | agent_addr = 'mock_addr:12345' 11 | framework_id = str(uuid.uuid4()) 12 | executor_id = str(uuid.uuid4()) 13 | env = { 14 | 'MESOS_LOCAL': 'true', 15 | 'MESOS_AGENT_ENDPOINT': agent_addr, 16 | 'MESOS_FRAMEWORK_ID': framework_id, 17 | 'MESOS_EXECUTOR_ID': executor_id, 18 | } 19 | mocker.patch('os.environ', env) 20 | exc = mocker.Mock() 21 | driver = MesosExecutorDriver(exc) 22 | driver._master = agent_addr 23 | assert driver.framework_id == dict(value=framework_id) 24 | assert driver.executor_id == dict(value=executor_id) 25 | assert -1e-5 < driver.grace_shutdown_period < 1e-5 26 | assert not driver.checkpoint 27 | assert driver.executor is exc 28 | 29 | req = driver.gen_request() 30 | parser = HttpParser(0) 31 | assert len(req) == parser.execute(req, len(req)) 32 | assert parser.is_headers_complete() 33 | assert parser.get_method() == 'POST' 34 | assert parser.get_url() == '/api/v1/executor' 35 | 36 | assert parser.is_partial_body() 37 | body = parser.recv_body() 38 | result = json.loads(body.decode('utf-8')) 39 | assert result == { 40 | 'type': 'SUBSCRIBE', 41 | 'framework_id': { 42 | 'value': framework_id, 43 | }, 44 | 'executor_id': { 45 | 'value': executor_id, 46 | }, 47 | 'subscribe': { 48 | 'unacknowledged_tasks': [], 49 | 'unacknowledged_updates': [], 50 | } 51 | } 52 | 53 | headers = {k.upper(): v for k, v in parser.get_headers().items()} 54 | assert headers == { 55 | 'HOST': agent_addr, 56 | 'CONTENT-TYPE': 'application/json', 57 | 'ACCEPT': 'application/json', 58 | 'CONNECTION': 'close', 59 | 'CONTENT-LENGTH': str(len(body)) 60 | } 61 | 62 | assert parser.is_message_complete() 63 | 64 | 65 | def test_send(mocker): 66 | agent_addr = 'mock_addr:12345' 67 | framework_id = str(uuid.uuid4()) 68 | executor_id = str(uuid.uuid4()) 69 | env = { 70 | 'MESOS_LOCAL': 'true', 71 | 'MESOS_AGENT_ENDPOINT': agent_addr, 72 | 'MESOS_FRAMEWORK_ID': framework_id, 73 | 'MESOS_EXECUTOR_ID': executor_id, 74 | } 75 | mocker.patch('os.environ', env) 76 | exc = mocker.Mock() 77 | driver = MesosExecutorDriver(exc) 78 | resp = mocker.Mock( 79 | status=200, 80 | read=mocker.Mock(return_value='{}') 81 | ) 82 | conn = mocker.Mock( 83 | getresponse=mocker.Mock(return_value=resp) 84 | ) 85 | driver._get_conn = mocker.Mock(return_value=conn) 86 | assert driver._send({}) == {} 87 | driver._get_conn.assert_called_once_with() 88 | conn.request.assert_called_once_with( 89 | 'POST', '/api/v1/executor', 90 | body=json.dumps({}).encode('utf-8'), 91 | headers={ 92 | 'Content-Type': 'application/json' 93 | } 94 | ) 95 | 96 | 97 | def test_send_status_update(mocker): 98 | agent_addr = 'mock_addr:12345' 99 | framework_id = str(uuid.uuid4()) 100 | executor_id = str(uuid.uuid4()) 101 | env = { 102 | 'MESOS_LOCAL': 'true', 103 | 'MESOS_AGENT_ENDPOINT': agent_addr, 104 | 'MESOS_FRAMEWORK_ID': framework_id, 105 | 'MESOS_EXECUTOR_ID': executor_id, 106 | } 107 | mocker.patch('os.environ', env) 108 | exc = mocker.Mock() 109 | driver = MesosExecutorDriver(exc) 110 | driver._send = mocker.Mock() 111 | status = {} 112 | driver.sendStatusUpdate(status) 113 | driver._send.assert_called_once_with({ 114 | 'type': 'UPDATE', 115 | 'framework_id': { 116 | 'value': framework_id, 117 | }, 118 | 'executor_id': { 119 | 'value': executor_id, 120 | }, 121 | 'update': { 122 | 'status': { 123 | 'timestamp': status['timestamp'], 124 | 'uuid': status['uuid'], 125 | 'source': 'SOURCE_EXECUTOR', 126 | } 127 | } 128 | }) 129 | 130 | 131 | def test_send_message(mocker): 132 | agent_addr = 'mock_addr:12345' 133 | framework_id = str(uuid.uuid4()) 134 | executor_id = str(uuid.uuid4()) 135 | env = { 136 | 'MESOS_LOCAL': 'true', 137 | 'MESOS_AGENT_ENDPOINT': agent_addr, 138 | 'MESOS_FRAMEWORK_ID': framework_id, 139 | 'MESOS_EXECUTOR_ID': executor_id, 140 | } 141 | mocker.patch('os.environ', env) 142 | exc = mocker.Mock() 143 | driver = MesosExecutorDriver(exc) 144 | driver._send = mocker.Mock() 145 | message = ''.join(random.choice(string.printable) 146 | for _ in range(random.randint(1, 100))) 147 | message = encode_data(message.encode('utf8')) 148 | driver.sendFrameworkMessage(message) 149 | driver._send.assert_called_once_with({ 150 | 'type': 'MESSAGE', 151 | 'framework_id': { 152 | 'value': framework_id, 153 | }, 154 | 'executor_id': { 155 | 'value': executor_id, 156 | }, 157 | 'message': { 158 | 'data': message, 159 | } 160 | }) 161 | 162 | 163 | def test_on_subscribed(mocker): 164 | agent_addr = 'mock_addr:12345' 165 | framework_id = str(uuid.uuid4()) 166 | executor_id = str(uuid.uuid4()) 167 | env = { 168 | 'MESOS_LOCAL': 'true', 169 | 'MESOS_AGENT_ENDPOINT': agent_addr, 170 | 'MESOS_FRAMEWORK_ID': framework_id, 171 | 'MESOS_EXECUTOR_ID': executor_id, 172 | } 173 | mocker.patch('os.environ', env) 174 | exc = mocker.Mock() 175 | driver = MesosExecutorDriver(exc) 176 | driver._started = True 177 | executor_info = { 178 | 'executor_id': { 179 | 'value': executor_id 180 | }, 181 | 'framework_id': { 182 | 'value': framework_id 183 | } 184 | } 185 | framework_info = { 186 | 'id': { 187 | 'value': framework_id 188 | } 189 | } 190 | event = { 191 | 'type': 'SUBSCRIBED', 192 | 'subscribed': { 193 | 'executor_info': executor_info, 194 | 'framework_info': framework_info, 195 | 'agent_info': {} 196 | } 197 | } 198 | driver.on_event(event) 199 | exc.registered.assert_called_once_with( 200 | driver, executor_info, framework_info, {}) 201 | 202 | 203 | def test_on_launch(mocker): 204 | agent_addr = 'mock_addr:12345' 205 | framework_id = str(uuid.uuid4()) 206 | executor_id = str(uuid.uuid4()) 207 | env = { 208 | 'MESOS_LOCAL': 'true', 209 | 'MESOS_AGENT_ENDPOINT': agent_addr, 210 | 'MESOS_FRAMEWORK_ID': framework_id, 211 | 'MESOS_EXECUTOR_ID': executor_id, 212 | } 213 | mocker.patch('os.environ', env) 214 | exc = mocker.Mock() 215 | driver = MesosExecutorDriver(exc) 216 | driver._started = True 217 | task_id = str(uuid.uuid4()) 218 | framework_info = { 219 | 'id': { 220 | 'value': framework_id 221 | } 222 | } 223 | task_info = { 224 | "task_id": { 225 | "value": task_id 226 | }, 227 | } 228 | event = { 229 | 'type': 'LAUNCH', 230 | 'launch': { 231 | 'framework_info': framework_info, 232 | 'task': task_info 233 | } 234 | } 235 | driver.on_event(event) 236 | exc.launchTask.assert_called_once_with(driver, task_info) 237 | 238 | 239 | def test_on_kill(mocker): 240 | agent_addr = 'mock_addr:12345' 241 | framework_id = str(uuid.uuid4()) 242 | executor_id = str(uuid.uuid4()) 243 | env = { 244 | 'MESOS_LOCAL': 'true', 245 | 'MESOS_AGENT_ENDPOINT': agent_addr, 246 | 'MESOS_FRAMEWORK_ID': framework_id, 247 | 'MESOS_EXECUTOR_ID': executor_id, 248 | } 249 | mocker.patch('os.environ', env) 250 | exc = mocker.Mock() 251 | driver = MesosExecutorDriver(exc) 252 | driver._started = True 253 | task_id = { 254 | 'value': str(uuid.uuid4()) 255 | } 256 | event = { 257 | 'type': 'KILL', 258 | 'kill': { 259 | 'task_id': task_id 260 | } 261 | } 262 | driver.on_event(event) 263 | exc.killTask.assert_called_once_with(driver, task_id) 264 | 265 | 266 | def test_on_acknowledged(mocker): 267 | agent_addr = 'mock_addr:12345' 268 | framework_id = str(uuid.uuid4()) 269 | executor_id = str(uuid.uuid4()) 270 | env = { 271 | 'MESOS_LOCAL': 'true', 272 | 'MESOS_AGENT_ENDPOINT': agent_addr, 273 | 'MESOS_FRAMEWORK_ID': framework_id, 274 | 'MESOS_EXECUTOR_ID': executor_id, 275 | } 276 | mocker.patch('os.environ', env) 277 | exc = mocker.Mock() 278 | driver = MesosExecutorDriver(exc) 279 | driver._started = True 280 | assert driver.updates == {} 281 | assert driver.tasks == {} 282 | tid = str(uuid.uuid4()) 283 | uid = uuid.uuid4() 284 | driver.updates[uid] = mocker.Mock() 285 | driver.tasks[tid] = mocker.Mock() 286 | event = { 287 | 'type': 'ACKNOWLEDGED', 288 | 'acknowledged': { 289 | 'task_id': { 290 | 'value': tid 291 | }, 292 | 'uuid': encode_data(uid.bytes) 293 | } 294 | } 295 | driver.on_event(event) 296 | assert driver.updates == {} 297 | assert driver.tasks == {} 298 | 299 | 300 | def test_on_message(mocker): 301 | agent_addr = 'mock_addr:12345' 302 | framework_id = str(uuid.uuid4()) 303 | executor_id = str(uuid.uuid4()) 304 | env = { 305 | 'MESOS_LOCAL': 'true', 306 | 'MESOS_AGENT_ENDPOINT': agent_addr, 307 | 'MESOS_FRAMEWORK_ID': framework_id, 308 | 'MESOS_EXECUTOR_ID': executor_id, 309 | } 310 | mocker.patch('os.environ', env) 311 | exc = mocker.Mock() 312 | driver = MesosExecutorDriver(exc) 313 | driver._started = True 314 | message = ''.join(random.choice(string.printable) 315 | for _ in range(random.randint(1, 100))) 316 | event = { 317 | 'type': 'MESSAGE', 318 | 'message': { 319 | 'data': message 320 | } 321 | } 322 | driver.on_event(event) 323 | exc.frameworkMessage.assert_called_once_with(driver, message) 324 | 325 | 326 | def test_on_error(mocker): 327 | agent_addr = 'mock_addr:12345' 328 | framework_id = str(uuid.uuid4()) 329 | executor_id = str(uuid.uuid4()) 330 | env = { 331 | 'MESOS_LOCAL': 'true', 332 | 'MESOS_AGENT_ENDPOINT': agent_addr, 333 | 'MESOS_FRAMEWORK_ID': framework_id, 334 | 'MESOS_EXECUTOR_ID': executor_id, 335 | } 336 | mocker.patch('os.environ', env) 337 | exc = mocker.Mock() 338 | driver = MesosExecutorDriver(exc) 339 | driver._started = True 340 | message = ''.join(random.choice(string.printable) 341 | for _ in range(random.randint(1, 100))) 342 | event = { 343 | 'type': 'ERROR', 344 | 'error': { 345 | 'message': message 346 | } 347 | } 348 | driver.on_event(event) 349 | exc.error.assert_called_once_with(driver, message) 350 | 351 | 352 | def test_on_shutdown(mocker): 353 | agent_addr = 'mock_addr:12345' 354 | framework_id = str(uuid.uuid4()) 355 | executor_id = str(uuid.uuid4()) 356 | env = { 357 | 'MESOS_LOCAL': 'true', 358 | 'MESOS_AGENT_ENDPOINT': agent_addr, 359 | 'MESOS_FRAMEWORK_ID': framework_id, 360 | 'MESOS_EXECUTOR_ID': executor_id, 361 | } 362 | mocker.patch('os.environ', env) 363 | exc = mocker.Mock() 364 | driver = MesosExecutorDriver(exc) 365 | driver._started = True 366 | event = { 367 | 'type': 'shutdown' 368 | } 369 | driver.on_event(event) 370 | exc.shutdown.assert_called_once_with(driver) 371 | -------------------------------------------------------------------------------- /tests/test_scheduler.py: -------------------------------------------------------------------------------- 1 | import json 2 | import uuid 3 | import random 4 | import string 5 | from six.moves import range 6 | from http_parser.http import HttpParser 7 | from pymesos import MesosSchedulerDriver, encode_data 8 | 9 | 10 | def test_gen_request(mocker): 11 | mock_addr = 'mock_addr:1234' 12 | sched = mocker.Mock() 13 | framework = { 14 | 'failover_timeout': 0 15 | } 16 | master = mocker.Mock() 17 | driver = MesosSchedulerDriver(sched, framework, master) 18 | driver._master = mock_addr 19 | req = driver.gen_request() 20 | parser = HttpParser(0) 21 | assert len(req) == parser.execute(req, len(req)) 22 | assert parser.is_headers_complete() 23 | assert parser.get_method() == 'POST' 24 | assert parser.get_url() == '/api/v1/scheduler' 25 | 26 | assert parser.is_partial_body() 27 | body = parser.recv_body() 28 | result = json.loads(body.decode('utf-8')) 29 | assert result['type'] == 'SUBSCRIBE' 30 | assert result['subscribe'] == { 31 | 'framework_info': framework 32 | } 33 | 34 | headers = {k.upper(): v for k, v in parser.get_headers().items()} 35 | assert headers == { 36 | 'HOST': mock_addr, 37 | 'CONTENT-TYPE': 'application/json', 38 | 'ACCEPT': 'application/json', 39 | 'CONNECTION': 'close', 40 | 'CONTENT-LENGTH': str(len(body)) 41 | } 42 | 43 | assert parser.is_message_complete() 44 | 45 | 46 | def test_send(mocker): 47 | sched = mocker.Mock() 48 | framework = mocker.Mock() 49 | master = mocker.Mock() 50 | driver = MesosSchedulerDriver(sched, framework, master) 51 | resp = mocker.Mock( 52 | status=200, 53 | read=mocker.Mock(return_value='{}') 54 | ) 55 | conn = mocker.Mock( 56 | getresponse=mocker.Mock(return_value=resp) 57 | ) 58 | driver._get_conn = mocker.Mock(return_value=conn) 59 | assert driver._send({}) == {} 60 | driver._get_conn.assert_called_once_with() 61 | conn.request.assert_called_once_with( 62 | 'POST', '/api/v1/scheduler', 63 | body=json.dumps({ 64 | }).encode('utf-8'), 65 | headers={ 66 | 'Content-Type': 'application/json' 67 | } 68 | ) 69 | 70 | 71 | def test_teardown(mocker): 72 | ID = str(uuid.uuid4()) 73 | sched = mocker.Mock() 74 | framework = {'id': {'value': ID}} 75 | master = mocker.Mock() 76 | driver = MesosSchedulerDriver(sched, framework, master) 77 | driver._send = mocker.Mock() 78 | driver.stream_id = str(uuid.uuid4()) 79 | assert driver.connected 80 | driver._teardown() 81 | driver._send.assert_called_once_with({ 82 | 'type': 'TEARDOWN', 83 | 'framework_id': { 84 | 'value': ID 85 | }, 86 | }) 87 | 88 | 89 | def test_accept_offers(mocker): 90 | ID = str(uuid.uuid4()) 91 | sched = mocker.Mock() 92 | framework = {'id': {'value': ID}} 93 | master = mocker.Mock() 94 | driver = MesosSchedulerDriver(sched, framework, master) 95 | driver._send = mocker.Mock() 96 | offer_ids = [str(uuid.uuid4()) for _ in range(random.randint(1, 10))] 97 | operations = [{ 98 | 'type': 'LAUNCH', 99 | 'launch': { 100 | 'task_infos': [ 101 | { 102 | 'name': '1', 103 | }, 104 | { 105 | 'name': '2', 106 | } 107 | ] 108 | } 109 | }] 110 | driver.acceptOffers(offer_ids, operations) 111 | driver._send.assert_not_called() 112 | driver._stream_id = 'a-stream-id' 113 | driver.acceptOffers(offer_ids, operations) 114 | driver._send.assert_called_once_with({ 115 | 'type': 'ACCEPT', 116 | 'framework_id': { 117 | 'value': ID 118 | }, 119 | 'accept': { 120 | 'offer_ids': offer_ids, 121 | 'operations': operations, 122 | } 123 | }) 124 | 125 | 126 | def test_launch_tasks(mocker): 127 | ID = str(uuid.uuid4()) 128 | sched = mocker.Mock() 129 | framework = {'id': {'value': ID}} 130 | master = mocker.Mock() 131 | driver = MesosSchedulerDriver(sched, framework, master) 132 | driver._send = mocker.Mock() 133 | offer_ids = [str(uuid.uuid4()) for _ in range(random.randint(1, 10))] 134 | tasks = [ 135 | { 136 | 'name': '1', 137 | }, 138 | { 139 | 'name': '2', 140 | } 141 | ] 142 | driver.launchTasks(offer_ids, tasks) 143 | driver._send.assert_not_called() 144 | driver._stream_id = 'a-stream-id' 145 | driver.launchTasks(offer_ids, tasks) 146 | driver._send.assert_called_once_with({ 147 | 'type': 'ACCEPT', 148 | 'framework_id': { 149 | 'value': ID 150 | }, 151 | 'accept': { 152 | 'offer_ids': offer_ids, 153 | 'operations': [{ 154 | 'type': 'LAUNCH', 155 | 'launch': { 156 | 'task_infos': tasks, 157 | } 158 | }] 159 | } 160 | }) 161 | 162 | 163 | def test_decline_offer(mocker): 164 | ID = str(uuid.uuid4()) 165 | sched = mocker.Mock() 166 | framework = {'id': {'value': ID}} 167 | master = mocker.Mock() 168 | driver = MesosSchedulerDriver(sched, framework, master) 169 | driver._send = mocker.Mock() 170 | offer_ids = [str(uuid.uuid4()) for _ in range(random.randint(1, 10))] 171 | driver.declineOffer(offer_ids) 172 | driver._send.assert_not_called() 173 | driver._stream_id = 'a-stream-id' 174 | driver.declineOffer(offer_ids) 175 | driver._send.assert_called_once_with({ 176 | 'type': 'DECLINE', 177 | 'framework_id': { 178 | 'value': ID 179 | }, 180 | 'decline': { 181 | 'offer_ids': offer_ids 182 | } 183 | }) 184 | 185 | 186 | def test_decline_inverse_offer(mocker): 187 | ID = str(uuid.uuid4()) 188 | sched = mocker.Mock() 189 | framework = {'id': {'value': ID}} 190 | master = mocker.Mock() 191 | driver = MesosSchedulerDriver(sched, framework, master) 192 | driver._send = mocker.Mock() 193 | offer_ids = [str(uuid.uuid4()) for _ in range(random.randint(1, 10))] 194 | driver.declineInverseOffer(offer_ids) 195 | driver._send.assert_not_called() 196 | driver._stream_id = 'a-stream-id' 197 | driver.declineInverseOffer(offer_ids) 198 | driver._send.assert_called_once_with({ 199 | 'type': 'DECLINE_INVERSE_OFFERS', 200 | 'framework_id': { 201 | 'value': ID 202 | }, 203 | 'decline_inverse_offers': { 204 | 'inverse_offer_ids': offer_ids 205 | } 206 | }) 207 | 208 | 209 | def test_revive_offers(mocker): 210 | ID = str(uuid.uuid4()) 211 | sched = mocker.Mock() 212 | framework = {'id': {'value': ID}} 213 | master = mocker.Mock() 214 | driver = MesosSchedulerDriver(sched, framework, master) 215 | driver._send = mocker.Mock() 216 | driver._stream_id = str(uuid.uuid4()) 217 | driver.reviveOffers() 218 | driver._send.assert_called_once_with({ 219 | 'type': 'REVIVE', 220 | 'framework_id': { 221 | 'value': ID 222 | }, 223 | }) 224 | 225 | 226 | def test_revive_offers_roles(mocker): 227 | ID = str(uuid.uuid4()) 228 | sched = mocker.Mock() 229 | framework = {'id': {'value': ID}} 230 | master = mocker.Mock() 231 | driver = MesosSchedulerDriver(sched, framework, master) 232 | driver._send = mocker.Mock() 233 | driver._stream_id = str(uuid.uuid4()) 234 | driver.reviveOffers(['role1', 'role2']) 235 | driver._send.assert_called_once_with({ 236 | 'type': 'REVIVE', 237 | 'framework_id': { 238 | 'value': ID 239 | }, 240 | 'revive': { 241 | 'roles': ['role1', 'role2'] 242 | } 243 | }) 244 | 245 | 246 | def test_suppress_offers(mocker): 247 | ID = str(uuid.uuid4()) 248 | sched = mocker.Mock() 249 | framework = {'id': {'value': ID}} 250 | master = mocker.Mock() 251 | driver = MesosSchedulerDriver(sched, framework, master) 252 | driver._send = mocker.Mock() 253 | driver._stream_id = str(uuid.uuid4()) 254 | driver.suppressOffers() 255 | driver._send.assert_called_once_with({ 256 | 'type': 'SUPPRESS', 257 | 'framework_id': { 258 | 'value': ID 259 | }, 260 | }) 261 | 262 | 263 | def test_suppress_offers_roles(mocker): 264 | ID = str(uuid.uuid4()) 265 | sched = mocker.Mock() 266 | framework = {'id': {'value': ID}} 267 | master = mocker.Mock() 268 | driver = MesosSchedulerDriver(sched, framework, master) 269 | driver._send = mocker.Mock() 270 | driver._stream_id = str(uuid.uuid4()) 271 | driver.suppressOffers(['role1', 'role2']) 272 | driver._send.assert_called_once_with({ 273 | 'type': 'SUPPRESS', 274 | 'framework_id': { 275 | 'value': ID 276 | }, 277 | 'suppress': { 278 | 'roles': ['role1', 'role2'] 279 | } 280 | }) 281 | 282 | 283 | def test_kill_task(mocker): 284 | ID = str(uuid.uuid4()) 285 | sched = mocker.Mock() 286 | framework = {'id': {'value': ID}} 287 | master = mocker.Mock() 288 | driver = MesosSchedulerDriver(sched, framework, master) 289 | driver._send = mocker.Mock() 290 | driver.killTask({"value": "my-task"}) 291 | driver._send.assert_not_called() 292 | driver._stream_id = str(uuid.uuid4()) 293 | driver.killTask({"value": "my-task"}) 294 | driver._send.assert_called_once_with({ 295 | 'type': 'KILL', 296 | 'framework_id': {'value': ID}, 297 | 'kill': { 298 | 'task_id': {'value': 'my-task'}, 299 | } 300 | }) 301 | 302 | 303 | def test_acknowledge_status_update(mocker): 304 | ID = str(uuid.uuid4()) 305 | sched = mocker.Mock() 306 | framework = {'id': {'value': ID}} 307 | master = mocker.Mock() 308 | driver = MesosSchedulerDriver(sched, framework, master) 309 | driver._send = mocker.Mock() 310 | agent_id = dict(value=str(uuid.uuid4())) 311 | task_id = dict(value=str(uuid.uuid4())) 312 | uid = encode_data(uuid.uuid4().bytes) 313 | status = { 314 | 'agent_id': agent_id, 315 | 'task_id': task_id, 316 | 'uuid': uid 317 | } 318 | driver.acknowledgeStatusUpdate(status) 319 | driver._send.assert_not_called() 320 | driver._stream_id = 'a-stream-id' 321 | driver.acknowledgeStatusUpdate(status) 322 | driver._send.assert_called_once_with({ 323 | 'type': 'ACKNOWLEDGE', 324 | 'framework_id': { 325 | 'value': ID 326 | }, 327 | 'acknowledge': { 328 | 'agent_id': agent_id, 329 | 'task_id': task_id, 330 | 'uuid': uid 331 | } 332 | }) 333 | 334 | 335 | def test_acknowledge_operation_status_update(mocker): 336 | ID = str(uuid.uuid4()) 337 | sched = mocker.Mock() 338 | framework = {'id': {'value': ID}} 339 | master = mocker.Mock() 340 | driver = MesosSchedulerDriver(sched, framework, master) 341 | driver._send = mocker.Mock() 342 | agent_id = dict(value=str(uuid.uuid4())) 343 | operation_id = dict(value=str(uuid.uuid4())) 344 | uid = encode_data(uuid.uuid4().bytes) 345 | status = { 346 | 'agent_id': agent_id, 347 | 'operation_id': operation_id, 348 | 'uuid': uid 349 | } 350 | driver.acknowledgeOperationStatusUpdate(status) 351 | driver._send.assert_not_called() 352 | driver._stream_id = 'a-stream-id' 353 | driver.acknowledgeOperationStatusUpdate(status) 354 | driver._send.assert_called_once_with({ 355 | 'type': 'ACKNOWLEDGE_OPERATION_STATUS', 356 | 'framework_id': { 357 | 'value': ID 358 | }, 359 | 'acknowledge_operation_status': { 360 | 'agent_id': agent_id, 361 | 'operation_id': operation_id, 362 | 'uuid': uid, 363 | } 364 | }) 365 | 366 | 367 | def test_reconcile_tasks(mocker): 368 | ID = str(uuid.uuid4()) 369 | sched = mocker.Mock() 370 | framework = {'id': {'value': ID}} 371 | master = mocker.Mock() 372 | driver = MesosSchedulerDriver(sched, framework, master) 373 | driver._send = mocker.Mock() 374 | task_ids = [str(uuid.uuid4()) for _ in range(random.randint(1, 10))] 375 | tasks = [ 376 | { 377 | 'task_id': { 378 | 'value': id 379 | } 380 | } 381 | for id in task_ids 382 | ] 383 | driver.reconcileTasks(tasks) 384 | driver._send.assert_not_called() 385 | driver._stream_id = 'a-stream-id' 386 | driver.reconcileTasks(tasks) 387 | driver._send.assert_called_once_with({ 388 | 'type': 'RECONCILE', 389 | 'framework_id': { 390 | 'value': ID 391 | }, 392 | 'reconcile': { 393 | 'tasks': tasks 394 | } 395 | }) 396 | 397 | 398 | def test_reconcile_operations(mocker): 399 | ID = str(uuid.uuid4()) 400 | sched = mocker.Mock() 401 | framework = {'id': {'value': ID}} 402 | master = mocker.Mock() 403 | driver = MesosSchedulerDriver(sched, framework, master) 404 | driver._send = mocker.Mock() 405 | op_ids = [str(uuid.uuid4()) for _ in range(random.randint(1, 10))] 406 | operations = [ 407 | { 408 | 'operation_id': { 409 | 'value': id 410 | } 411 | } 412 | for id in op_ids 413 | ] 414 | driver.reconcileOperations(operations) 415 | driver._send.assert_not_called() 416 | driver._stream_id = 'a-stream-id' 417 | driver.reconcileOperations(operations) 418 | driver._send.assert_called_once_with({ 419 | 'type': 'RECONCILE_OPERATIONS', 420 | 'framework_id': { 421 | 'value': ID 422 | }, 423 | 'reconcile_operations': { 424 | 'operations': operations 425 | } 426 | }) 427 | 428 | 429 | def test_send_framework_message(mocker): 430 | ID = str(uuid.uuid4()) 431 | sched = mocker.Mock() 432 | framework = {'id': {'value': ID}} 433 | master = mocker.Mock() 434 | driver = MesosSchedulerDriver(sched, framework, master) 435 | driver._send = mocker.Mock() 436 | executor_id = {'value': str(uuid.uuid4())} 437 | agent_id = {'value': str(uuid.uuid4())} 438 | message = ''.join(random.choice(string.printable) 439 | for _ in range(random.randint(1, 100))) 440 | message = encode_data(message.encode('utf-8')) 441 | driver.sendFrameworkMessage(executor_id, agent_id, message) 442 | driver._send.assert_not_called() 443 | driver._stream_id = 'a-stream-id' 444 | driver.sendFrameworkMessage(executor_id, agent_id, message) 445 | driver._send.assert_called_once_with({ 446 | 'type': 'MESSAGE', 447 | 'framework_id': { 448 | 'value': ID 449 | }, 450 | 'message': { 451 | 'agent_id': agent_id, 452 | 'executor_id': executor_id, 453 | 'data': message, 454 | } 455 | }) 456 | 457 | 458 | def test_request_resources(mocker): 459 | ID = str(uuid.uuid4()) 460 | sched = mocker.Mock() 461 | framework = {'id': {'value': ID}} 462 | master = mocker.Mock() 463 | driver = MesosSchedulerDriver(sched, framework, master) 464 | driver._send = mocker.Mock() 465 | requests = [{ 466 | 'agent_id': {'value': str(uuid.uuid4())}, 467 | 'resources': {} 468 | } for _ in range(random.randint(1, 10))] 469 | driver.requestResources(requests) 470 | driver._send.assert_not_called() 471 | driver._stream_id = 'a-stream-id' 472 | driver.requestResources(requests) 473 | driver._send.assert_called_once_with({ 474 | 'type': 'REQUEST', 475 | 'framework_id': { 476 | 'value': ID 477 | }, 478 | 'request': { 479 | 'requests': requests 480 | } 481 | }) 482 | 483 | 484 | def test_on_subscribed(mocker): 485 | sched = mocker.Mock() 486 | framework = {} 487 | master = mocker.Mock() 488 | driver = MesosSchedulerDriver(sched, framework, master) 489 | driver.version = '1.0.0' 490 | driver._started = True 491 | driver._master = 'mock_addr:12345' 492 | framework_id = { 493 | 'value': str(uuid.uuid4()) 494 | } 495 | 496 | event = { 497 | 'type': 'SUBSCRIBED', 498 | 'subscribed': { 499 | 'framework_id': framework_id 500 | } 501 | } 502 | driver.on_event(event) 503 | sched.registered.assert_called_once_with(driver, framework_id, { 504 | 'hostname': 'mock_addr', 505 | 'port': 12345, 506 | 'version': '1.0.0' 507 | }) 508 | 509 | 510 | def test_on_offers(mocker): 511 | ID = str(uuid.uuid4()) 512 | sched = mocker.Mock() 513 | framework = {'id': {'value': ID}} 514 | master = mocker.Mock() 515 | driver = MesosSchedulerDriver(sched, framework, master) 516 | driver._started = True 517 | offers = [{ 518 | 'offer_id': {'value': str(uuid.uuid4())} 519 | } for _ in range(random.randint(1, 10))] 520 | event = { 521 | 'type': 'OFFERS', 522 | 'offers': { 523 | 'offers': offers 524 | } 525 | } 526 | driver.on_event(event) 527 | sched.resourceOffers.assert_called_once_with(driver, offers) 528 | sched.inverseOffers.assert_not_called() 529 | 530 | 531 | def test_on_offers_inverse(mocker): 532 | ID = str(uuid.uuid4()) 533 | sched = mocker.Mock() 534 | framework = {'id': {'value': ID}} 535 | master = mocker.Mock() 536 | driver = MesosSchedulerDriver(sched, framework, master) 537 | driver._started = True 538 | offers = [{ 539 | 'offer_id': {'value': str(uuid.uuid4())} 540 | } for _ in range(random.randint(1, 10))] 541 | event = { 542 | 'type': 'OFFERS', 543 | 'offers': { 544 | 'inverse_offers': offers 545 | } 546 | } 547 | driver.on_event(event) 548 | sched.resourceOffers.assert_not_called() 549 | sched.inverseOffers.assert_called_once_with(driver, offers) 550 | 551 | 552 | def test_on_rescind(mocker): 553 | ID = str(uuid.uuid4()) 554 | sched = mocker.Mock() 555 | framework = {'id': {'value': ID}} 556 | master = mocker.Mock() 557 | driver = MesosSchedulerDriver(sched, framework, master) 558 | driver._started = True 559 | offer_id = {'value': str(uuid.uuid4())} 560 | event = { 561 | 'type': 'RESCIND', 562 | 'rescind': { 563 | 'offer_id': offer_id 564 | } 565 | } 566 | driver.on_event(event) 567 | sched.offerRescinded.assert_called_once_with(driver, offer_id) 568 | 569 | 570 | def test_on_message(mocker): 571 | ID = str(uuid.uuid4()) 572 | sched = mocker.Mock() 573 | framework = {'id': {'value': ID}} 574 | master = mocker.Mock() 575 | driver = MesosSchedulerDriver(sched, framework, master) 576 | driver._started = True 577 | executor_id = {'value': str(uuid.uuid4())} 578 | agent_id = {'value': str(uuid.uuid4())} 579 | message = ''.join(random.choice(string.printable) 580 | for _ in range(random.randint(1, 100))) 581 | data = encode_data(message.encode('utf8')) 582 | 583 | event = { 584 | 'type': 'MESSAGE', 585 | 'message': { 586 | 'executor_id': executor_id, 587 | 'agent_id': agent_id, 588 | 'data': data 589 | } 590 | } 591 | driver.on_event(event) 592 | sched.frameworkMessage.assert_called_once_with(driver, executor_id, 593 | agent_id, data) 594 | 595 | 596 | def test_on_failure(mocker): 597 | ID = str(uuid.uuid4()) 598 | sched = mocker.Mock() 599 | framework = {'id': {'value': ID}} 600 | master = mocker.Mock() 601 | driver = MesosSchedulerDriver(sched, framework, master) 602 | driver._started = True 603 | executor_id = dict(value=str(uuid.uuid4())) 604 | agent_id = dict(value=str(uuid.uuid4())) 605 | status = random.randint(0, 256) 606 | event = { 607 | 'type': 'FAILURE', 608 | 'failure': { 609 | 'executor_id': executor_id, 610 | 'agent_id': agent_id, 611 | 'status': status 612 | } 613 | } 614 | driver.on_event(event) 615 | sched.executorLost.assert_called_once_with(driver, executor_id, 616 | agent_id, status) 617 | 618 | event = { 619 | 'type': 'FAILURE', 620 | 'failure': { 621 | 'agent_id': agent_id, 622 | } 623 | } 624 | driver.on_event(event) 625 | sched.slaveLost.assert_called_once_with(driver, agent_id) 626 | 627 | 628 | def test_on_error(mocker): 629 | ID = str(uuid.uuid4()) 630 | sched = mocker.Mock() 631 | framework = {'id': {'value': ID}} 632 | master = mocker.Mock() 633 | driver = MesosSchedulerDriver(sched, framework, master) 634 | driver._started = True 635 | msg = 'error message' 636 | event = { 637 | 'type': 'ERROR', 638 | 'error': { 639 | 'message': msg 640 | } 641 | } 642 | driver.on_event(event) 643 | sched.error.assert_called_once_with(driver, msg) 644 | -------------------------------------------------------------------------------- /tox.ini: -------------------------------------------------------------------------------- 1 | [tox] 2 | envlist = py27, py37, py38, pypy, flake8 3 | 4 | [testenv] 5 | commands = {envpython} setup.py test 6 | deps = -rrequirements.txt 7 | 8 | [testenv:flake8] 9 | basepython=python 10 | deps=flake8 11 | commands=flake8 pymesos tests examples 12 | 13 | [pep8] 14 | exclude = pymesos/interface.py 15 | 16 | [flake8] 17 | exclude = pymesos/interface.py,venv/ 18 | 19 | [pytest] 20 | testpaths = tests 21 | addopts = --cov=pymesos --cov-report term-missing 22 | --------------------------------------------------------------------------------