Hi Sungwoo,

Sorry for the late reply. Reusing a committed shuffleId does not work in
current architecture, even after calling unregisterShuffle in
LifecycleManager,
because the cleanup of metadata is delayed and not guaranteed. It will
be more complicated when we consider graceful restart of workers.

If we want to reuse the shuffleId, we need to redesign the whole picture.

Thanks,
Keyong Zhou

Sungwoo Park <o...@pl.postech.ac.kr> 于2023年10月2日周一 13:23写道:

> Hi Keyong,
>
> Instead of picking up a new shuffleId, can we reuse an existing shuffleId
> after unregistering it? If the following plan worked, it would further
> simplify the implementation:
>
> 1. Downstream tasks fail because of read failures.
> 2. All active downstream tasks are killed, so the shuffleId is not used.
> 3. An upstream vertex unregisters the shuffleId.
> 4. The upstream vertex is re-executed normally. This re-execution
> automatically registers the same shuffleId.
>
> In summary, we would like to go back in time before the upstream vertex
> started by cleaning up the shuffleId. Could you please give a comment on
> this plan?
>
> Thank you,
>
> --- Sungwoo
>
> On Sat, 30 Sep 2023, Keyong Zhou wrote:
>
> > Hi Sungwoo,
> >
> > I think your approach works with current architecture of Celeborn,
> > and interpreting IOException when reading as read failure makes
> > sense. Currently only when CommitFiles fails will LifecycleManager
> > announce data lost.
> >
> > Thanks,
> > Keyong Zhou
> >
> > Sungwoo Park <o...@pl.postech.ac.kr> 于2023年9月29日周五 22:05?道:
> >
> >>> Since the partition split has a good chance to contain data from almost
> >> all
> >>> upstream
> >>> mapper tasks, the cost of re-computing all upstream tasks may have
> little
> >>> difference
> >>> to re-computing the actual mapper tasks in most cases. Of course it's
> not
> >>> always true.
> >>>
> >>> To change from 'complete' to 'incomplete' also needs to refactor
> Worker's
> >>> logic, which
> >>> currently assumes that the succeeded attempts will not be changed after
> >>> final committing
> >>> files.
> >>>
> >>>> a subset of succeeded attempts. In Erik's proposal, the whole upstream
> >>>> stage will be rerun when data lost.
> >>
> >> Thank you for your response --- things are now much clearer.
> >>
> >> From your comments shown above, let me assume that:
> >>
> >> 1. The whole upstream stage is rerun in the case of read failure.
> >>
> >> 2. Currently it is not easy to change the state of a shuffleId from
> >> 'complete' to 'incomplete'.
> >>
> >> Then, for Celeborn-MR3, I would like to experiment with the following
> >> approach:
> >>
> >> 1. If read failures occur for shuffleId #1, we pick up a new shuffleId
> >> #2.
> >>
> >> 2. The upstream stage (or Vertex in the case of MR3) re-executes all
> tasks
> >> again, but writes the output to shuffleId #2.
> >>
> >> 3. Tasks in the downstream stage re-try by reading from shuffleId #2.
> >>
> >> Do you think this approach makes sense under the current architecture of
> >> Celeborn? If this approach is feasible, MR3 only needs to be notified of
> >> read failures due to lost data by Celeborn ShuffleClient. Or, we could
> >> just interpret IOException from Celeborn ShuffleClient as read failures,
> >> in which case we can implement stage recompute without requiring any
> >> extension of Celeborn. (However, it would be great if Celeborn
> >> ShuffleClient could announce lost data explicitly.)
> >>
> >> An industrial user of Hive-MR3-Celeborn is trying hard to save disk
> usage
> >> on Celeborn workers (which use SSDs of limited capacity), so stage
> >> recompute would be a great new feature to them.
> >>
> >> Thank you,
> >>
> >> --- Sungwoo
> >>
> >>
> >>
> >

Reply via email to