def run_task(task_data): try: gc.disable() task, ntry = cPickle.loads(decompress(task_data)) setproctitle('dpark worker %s: run task %s' % (Script, task)) Accumulator.clear() result = task.run(ntry) accUpdate = Accumulator.values() if marshalable(result): flag, data = 0, marshal.dumps(result) else: flag, data = 1, cPickle.dumps(result, -1) data = compress(data) if len(data) > TASK_RESULT_LIMIT: path = LocalFileShuffle.getOutputFile(0, ntry, task.id, len(data)) f = open(path, 'w') f.write(data) f.close() data = '/'.join([LocalFileShuffle.getServerUri()] + path.split('/')[-3:]) flag += 2 return mesos_pb2.TASK_FINISHED, cPickle.dumps((Success(), (flag, data), accUpdate), -1) except FetchFailed, e: return mesos_pb2.TASK_FAILED, cPickle.dumps((e, None, None), -1)
def taskFinished(self, tid, tried, result, update): i = self.tidToIndex[tid] self.finished[i] = True self.tasksFinished += 1 task = self.tasks[i] task.used += time.time() - task.start self.total_used += task.used if getattr(sys.stderr, 'isatty', lambda: False)(): title = 'Job %d: task %s finished in %.1fs (%d/%d) ' % ( self.id, tid, task.used, self.tasksFinished, self.numTasks) logger.info( 'Task %s finished in %.1fs (%d/%d)' ' \x1b]2;%s\x07\x1b[1A', tid, task.used, self.tasksFinished, self.numTasks, title) from dpark.schedule import Success self.sched.taskEnded(task, Success(), result, update) for t in range(task.tried): if t + 1 != tried: self.sched.killTask(self.id, task.id, t + 1) if self.tasksFinished == self.numTasks: ts = [t.used for t in self.tasks] tried = [t.tried for t in self.tasks] logger.info( 'Job %d finished in %.1fs: min=%.1fs, ' 'avg=%.1fs, max=%.1fs, maxtry=%d', self.id, time.time() - self.start, min(ts), sum(ts) / len(ts), max(ts), max(tried)) self.sched.jobFinished(self)
def run_task(task, ntry): try: setproctitle('dpark worker %s: run task %s' % (Script, task)) Accumulator.clear() gc.disable() result = task.run(ntry) accUpdate = Accumulator.values() if marshalable(result): flag, data = 0, marshal.dumps(result) else: flag, data = 1, cPickle.dumps(result, -1) data = compress(data) if len(data) > TASK_RESULT_LIMIT: workdir = env.get('WORKDIR') name = 'task_%s_%s.result' % (task.id, ntry) path = os.path.join(workdir, name) f = open(path, 'w') f.write(data) f.close() data = LocalFileShuffle.getServerUri() + '/' + name flag += 2 return mesos_pb2.TASK_FINISHED, cPickle.dumps( (task.id, Success(), (flag, data), accUpdate), -1) except Exception, e: import traceback msg = traceback.format_exc() return mesos_pb2.TASK_FAILED, cPickle.dumps( (task.id, OtherFailure(msg), None, None), -1)
def run_task(task, aid): try: setproctitle('dpark worker %s: run task %s' % (Script, task)) Accumulator.clear() result = task.run(aid) accUpdate = Accumulator.values() try: flag, data = 0, marshal.dumps(result) except ValueError: flag, data = 1, cPickle.dumps(result) if len(data) > TASK_RESULT_LIMIT: workdir = env.get('WORKDIR') path = os.path.join(workdir, str(task.id) + '.result') with open(path, 'w') as f: f.write(data) data = path flag += 2 setproctitle('dpark worker: idle') return mesos_pb2.TASK_FINISHED, cPickle.dumps( (task.id, Success(), (flag, data), accUpdate), -1) except Exception, e: import traceback msg = traceback.format_exc() setproctitle('dpark worker: idle') return mesos_pb2.TASK_FAILED, cPickle.dumps( (task.id, OtherFailure(msg), None, None), -1)
def _task_finished(self, task_id, num_try, result, update, stats): i = self.tidToIndex[task_id] self.finished[i] = True self.tasksFinished += 1 task = self.tasks[i] hostname = self.id_retry_host[(task.id, num_try)] \ if (task.id, num_try) in self.id_retry_host else task.host task.time_used += time.time() - task.start self.total_time_used += task.time_used if getattr(self.sched, 'color', False): title = 'taskset %s: task %s finished in %.1fs (%d/%d) ' % ( self.id, task_id, task.time_used, self.tasksFinished, self.numTasks) msg = '\x1b]2;%s\x07\x1b[1A' % title logger.info(msg) from dpark.schedule import Success self.sched.taskEnded(task, Success(), result, update, stats) self.running_hosts[i] = [] self.task_host_manager.task_succeed(task.id, hostname, Success()) for t in range(task.num_try): if t + 1 != num_try: self.sched.killTask(task.id, t + 1) if self.tasksFinished == self.numTasks: ts = [t.time_used for t in self.tasks] num_try = [t.num_try for t in self.tasks] elasped = time.time() - self.start logger.info( 'taskset %s finished in %.1fs: min=%.1fs, ' 'avg=%.1fs, max=%.1fs, maxtry=%d, speedup=%.1f, local=%.1f%%', self.id, elasped, min(ts), sum(ts) / len(ts), max(ts), max(num_try), self.total_time_used / elasped, len(self.task_local_set) * 100. / len(self.tasks)) self.sched.tasksetFinished(self)
def test_task_host_manager(self): manager = TaskHostManager() manager.register_host('fake1', purge_elapsed=1) manager.register_host('fake2', purge_elapsed=1) manager.register_host('fake3', purge_elapsed=1) host_offers = {'fake1': (1, None), 'fake2': (2, None), 'fake3': (3, None)} manager.task_failed(1, 'fake2', OtherFailure('Mock failed')) assert manager.offer_choice(1, host_offers, ['fake3'])[0] == 1 time.sleep(1) manager.task_failed(1, 'fake1', OtherFailure('Mock failed')) assert manager.offer_choice(1, host_offers, [])[0] == 3 assert manager.offer_choice(1, host_offers, ['fake3'])[0] is None manager.task_succeed(2, 'fake2', Success()) assert manager.offer_choice(1, host_offers, ['fake3'])[0] is None time.sleep(1) assert manager.offer_choice(1, host_offers, ['fake3'])[0] == 2
def run_task(task_data): try: gc.disable() task, task_try_id = loads(decompress(task_data)) ttid = TTID(task_try_id) Accumulator.clear() result = task.run(ttid.ttid) env.task_stats.bytes_max_rss = resource.getrusage( resource.RUSAGE_SELF).ru_maxrss * 1024 accUpdate = Accumulator.values() MutableDict.flush() if marshalable(result): try: flag, data = 0, marshal.dumps(result) except Exception: flag, data = 1, cPickle.dumps(result, -1) else: flag, data = 1, cPickle.dumps(result, -1) data = compress(data) if len(data) > TASK_RESULT_LIMIT: path = LocalFileShuffle.getOutputFile(0, task.id, ttid.task_try, len(data)) f = open(path, 'wb') f.write(data) f.close() data = '/'.join([LocalFileShuffle.getServerUri()] + path.split('/')[-3:]) flag += 2 return 'TASK_FINISHED', cPickle.dumps( (Success(), (flag, data), accUpdate, env.task_stats), -1) except FetchFailed as e: return 'TASK_FAILED', cPickle.dumps((e, None, None, None), -1) except: import traceback msg = traceback.format_exc() return 'TASK_FAILED', cPickle.dumps( (OtherFailure(msg), None, None, None), -1) finally: gc.collect() gc.enable()
def run_task(task_data): try: gc.disable() task, ntry = loads(decompress(task_data)) Accumulator.clear() result = task.run(ntry) accUpdate = Accumulator.values() MutableDict.flush() if marshalable(result): try: flag, data = 0, marshal.dumps(result) except Exception as e: flag, data = 1, cPickle.dumps(result, -1) else: flag, data = 1, cPickle.dumps(result, -1) data = compress(data) if len(data) > TASK_RESULT_LIMIT: path = LocalFileShuffle.getOutputFile(0, ntry, task.id, len(data)) f = open(path, 'w') f.write(data) f.close() data = '/'.join( [LocalFileShuffle.getServerUri()] + path.split('/')[-3:] ) flag += 2 return 'TASK_FINISHED', cPickle.dumps( (Success(), (flag, data), accUpdate), -1) except FetchFailed as e: return 'TASK_FAILED', cPickle.dumps((e, None, None), -1) except: import traceback msg = traceback.format_exc() return 'TASK_FAILED', cPickle.dumps( (OtherFailure(msg), None, None), -1) finally: close_mfs() gc.collect() gc.enable()
def taskFinished(self, tid, tried, result, update): i = self.tidToIndex[tid] self.finished[i] = True self.tasksFinished += 1 task = self.tasks[i] task.used += time.time() - task.start self.total_used += task.used if sys.stderr.isatty(): title = "Job %d: task %s finished in %.1fs (%d/%d) " % ( self.id, tid, task.used, self.tasksFinished, self.numTasks) logger.info( "Task %s finished in %.1fs (%d/%d) \x1b]2;%s\x07\x1b[1A", tid, task.used, self.tasksFinished, self.numTasks, title) from dpark.schedule import Success self.sched.taskEnded(task, Success(), result, update) for t in range(task.tried): if t + 1 != tried: self.sched.killTask(self.id, task.id, t + 1) if self.tasksFinished == self.numTasks: ts = [t.used for t in self.tasks] tried = [t.tried for t in self.tasks] logger.info( "Job %d finished in %.1fs: min=%.1fs, avg=%.1fs, max=%.1fs, maxtry=%d", self.id, time.time() - self.start, min(ts), sum(ts) / len(ts), max(ts), max(tried)) from dpark.accumulator import LocalReadBytes, RemoteReadBytes lb, rb = LocalReadBytes.reset(), RemoteReadBytes.reset() if rb > 0: logger.info("read %s (%d%% localized)", readable(lb + rb), lb * 100 / (rb + lb)) self.sched.jobFinished(self)
else: flag, data = 1, cPickle.dumps(result, -1) data = compress(data) if len(data) > TASK_RESULT_LIMIT: path = LocalFileShuffle.getOutputFile(0, ntry, task.id, len(data)) f = open(path, 'w') f.write(data) f.close() data = '/'.join([LocalFileShuffle.getServerUri()] + path.split('/')[-3:]) flag += 2 return mesos_pb2.TASK_FINISHED, cPickle.dumps( (Success(), (flag, data), accUpdate), -1) except FetchFailed, e: return mesos_pb2.TASK_FAILED, cPickle.dumps((e, None, None), -1) except: import traceback msg = traceback.format_exc() return mesos_pb2.TASK_FAILED, cPickle.dumps( (OtherFailure(msg), None, None), -1) finally: gc.collect() gc.enable() def init_env(args): env.start(False, args)
except Exception, e: flag, data = 1, cPickle.dumps(result, -1) else: flag, data = 1, cPickle.dumps(result, -1) data = compress(data) if len(data) > TASK_RESULT_LIMIT: path = LocalFileShuffle.getOutputFile(0, ntry, task.id, len(data)) f = open(path, 'w') f.write(data) f.close() data = '/'.join([LocalFileShuffle.getServerUri()] + path.split('/')[-3:]) flag += 2 return mesos_pb2.TASK_FINISHED, cPickle.dumps((Success(), (flag, data), accUpdate), -1) except FetchFailed, e: return mesos_pb2.TASK_FAILED, cPickle.dumps((e, None, None), -1) except : import traceback msg = traceback.format_exc() return mesos_pb2.TASK_FAILED, cPickle.dumps((OtherFailure(msg), None, None), -1) finally: setproctitle('dpark worker: idle') gc.collect() gc.enable() def init_env(args): setproctitle('dpark worker: idle') env.start(False, args)