|
| 1 | +# |
| 2 | +# Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | +# contributor license agreements. See the NOTICE file distributed with |
| 4 | +# this work for additional information regarding copyright ownership. |
| 5 | +# The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | +# (the "License"); you may not use this file except in compliance with |
| 7 | +# 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 | + |
| 18 | +from collections import namedtuple |
| 19 | + |
| 20 | +__all__ = ["SparkJobInfo", "SparkStageInfo", "StatusTracker"] |
| 21 | + |
| 22 | + |
| 23 | +class SparkJobInfo(namedtuple("SparkJobInfo", "jobId stageIds status")): |
| 24 | + """ |
| 25 | + Exposes information about Spark Jobs. |
| 26 | + """ |
| 27 | + |
| 28 | + |
| 29 | +class SparkStageInfo(namedtuple("SparkStageInfo", |
| 30 | + "stageId currentAttemptId name numTasks numActiveTasks " |
| 31 | + "numCompletedTasks numFailedTasks")): |
| 32 | + """ |
| 33 | + Exposes information about Spark Stages. |
| 34 | + """ |
| 35 | + |
| 36 | + |
| 37 | +class StatusTracker(object): |
| 38 | + """ |
| 39 | + Low-level status reporting APIs for monitoring job and stage progress. |
| 40 | +
|
| 41 | + These APIs intentionally provide very weak consistency semantics; |
| 42 | + consumers of these APIs should be prepared to handle empty / missing |
| 43 | + information. For example, a job's stage ids may be known but the status |
| 44 | + API may not have any information about the details of those stages, so |
| 45 | + `getStageInfo` could potentially return `None` for a valid stage id. |
| 46 | +
|
| 47 | + To limit memory usage, these APIs only provide information on recent |
| 48 | + jobs / stages. These APIs will provide information for the last |
| 49 | + `spark.ui.retainedStages` stages and `spark.ui.retainedJobs` jobs. |
| 50 | + """ |
| 51 | + def __init__(self, jtracker): |
| 52 | + self._jtracker = jtracker |
| 53 | + |
| 54 | + def getJobIdsForGroup(self, jobGroup=None): |
| 55 | + """ |
| 56 | + Return a list of all known jobs in a particular job group. If |
| 57 | + `jobGroup` is None, then returns all known jobs that are not |
| 58 | + associated with a job group. |
| 59 | +
|
| 60 | + The returned list may contain running, failed, and completed jobs, |
| 61 | + and may vary across invocations of this method. This method does |
| 62 | + not guarantee the order of the elements in its result. |
| 63 | + """ |
| 64 | + return list(self._jtracker.getJobIdsForGroup(jobGroup)) |
| 65 | + |
| 66 | + def getActiveStageIds(self): |
| 67 | + """ |
| 68 | + Returns an array containing the ids of all active stages. |
| 69 | + """ |
| 70 | + return sorted(list(self._jtracker.getActiveStageIds())) |
| 71 | + |
| 72 | + def getActiveJobsIds(self): |
| 73 | + """ |
| 74 | + Returns an array containing the ids of all active jobs. |
| 75 | + """ |
| 76 | + return sorted((list(self._jtracker.getActiveJobIds()))) |
| 77 | + |
| 78 | + def getJobInfo(self, jobId): |
| 79 | + """ |
| 80 | + Returns a :class:`SparkJobInfo` object, or None if the job info |
| 81 | + could not be found or was garbage collected. |
| 82 | + """ |
| 83 | + job = self._jtracker.getJobInfo(jobId) |
| 84 | + if job is not None: |
| 85 | + return SparkJobInfo(jobId, job.stageIds(), str(job.status())) |
| 86 | + |
| 87 | + def getStageInfo(self, stageId): |
| 88 | + """ |
| 89 | + Returns a :class:`SparkStageInfo` object, or None if the stage |
| 90 | + info could not be found or was garbage collected. |
| 91 | + """ |
| 92 | + stage = self._jtracker.getStageInfo(stageId) |
| 93 | + if stage is not None: |
| 94 | + # TODO: fetch them in batch for better performance |
| 95 | + attrs = [getattr(stage, f)() for f in SparkStageInfo._fields[1:]] |
| 96 | + return SparkStageInfo(stageId, *attrs) |
0 commit comments