Thanks Mark and Jeff

On Wed, Mar 16, 2016 at 7:11 AM, Mark Hamstra <m...@clearstorydata.com>
wrote:

> Looks to me like the one remaining Stage would execute 19788 Task if all
> of those Tasks succeeded on the first try; but because of retries, 19841
> Tasks were actually executed.  Meanwhile, there were 41405 Tasks in the the
> 163 Stages that were skipped.
>
> I think -- but the Spark UI's accounting may not be 100% accurate and bug
> free.
>
> On Tue, Mar 15, 2016 at 6:34 PM, Prabhu Joseph <prabhujose.ga...@gmail.com
> > wrote:
>
>> Okay, so out of 164 stages, is 163 are skipped. And how 41405 tasks are
>> skipped if the total is only 19788.
>>
>> On Wed, Mar 16, 2016 at 6:31 AM, Mark Hamstra <m...@clearstorydata.com>
>> wrote:
>>
>>> It's not just if the RDD is explicitly cached, but also if the map
>>> outputs for stages have been materialized into shuffle files and are still
>>> accessible through the map output tracker.  Because of that, explicitly
>>> caching RDD actions often gains you little or nothing, since even without a
>>> call to cache() or persist() the prior computation will largely be reused
>>> and stages will show up as skipped -- i.e. no need to recompute that stage.
>>>
>>> On Tue, Mar 15, 2016 at 5:50 PM, Jeff Zhang <zjf...@gmail.com> wrote:
>>>
>>>> If RDD is cached, this RDD is only computed once and the stages for
>>>> computing this RDD in the following jobs are skipped.
>>>>
>>>>
>>>> On Wed, Mar 16, 2016 at 8:14 AM, Prabhu Joseph <
>>>> prabhujose.ga...@gmail.com> wrote:
>>>>
>>>>> Hi All,
>>>>>
>>>>>
>>>>> Spark UI Completed Jobs section shows below information, what is the
>>>>> skipped value shown for Stages and Tasks below.
>>>>>
>>>>> Job_ID    Description    Submitted
>>>>> Duration           Stages (Succeeded/Total)    Tasks (for all stages):
>>>>> Succeeded/Total
>>>>>
>>>>> 11             count          2016/03/14 15:35:32      1.4
>>>>> min             164/164 * (163 skipped)   *            19841/19788
>>>>> *(41405 skipped)*
>>>>> Thanks,
>>>>> Prabhu Joseph
>>>>>
>>>>
>>>>
>>>>
>>>> --
>>>> Best Regards
>>>>
>>>> Jeff Zhang
>>>>
>>>
>>>
>>
>

Reply via email to