spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Matt Cheah (JIRA)" <j...@apache.org>
Subject [jira] [Created] (SPARK-25299) Use distributed storage for persisting shuffle data
Date Sat, 01 Sep 2018 00:26:00 GMT
Matt Cheah created SPARK-25299:
----------------------------------

             Summary: Use distributed storage for persisting shuffle data
                 Key: SPARK-25299
                 URL: https://issues.apache.org/jira/browse/SPARK-25299
             Project: Spark
          Issue Type: New Feature
          Components: Shuffle
    Affects Versions: 2.4.0
            Reporter: Matt Cheah


In Spark, the shuffle primitive requires Spark executors to persist data to the local disk
of the worker nodes. If executors crash, the external shuffle service can continue to serve
the shuffle data that was written beyond the lifetime of the executor itself. In YARN, Mesos,
and Standalone mode, the external shuffle service is deployed on every worker node. The shuffle
service shares local disk with the executors that run on its node.

There are some shortcomings with the way shuffle is fundamentally implemented right now. Particularly:
 * If any external shuffle service process or node becomes unavailable, all applications that
had an executor that ran on that node must recompute the shuffle blocks that were lost.
 * Similarly to the above, the external shuffle service must be kept running at all times,
which may waste resources when no applications are using that shuffle service node.
 * Mounting local storage can prevent users from taking advantage of desirable isolation
benefits from using containerized environments, like Kubernetes. We had an external shuffle
service implementation in an early prototype of the Kubernetes backend, but it was rejected
due to its strict requirement to be able to mount hostPath volumes or other persistent volume
setups.

In the following [architecture discussion document|https://docs.google.com/document/d/1uCkzGGVG17oGC6BJ75TpzLAZNorvrAU3FRd2X-rVHSM/edit#heading=h.btqugnmt2h40] (note:
_not_ an SPIP), we brainstorm various high level architectures for improving the external
shuffle service in a way that addresses the above problems. The purpose of this umbrella JIRA
is to promote additional discussion on how we can approach these problems, both at the architecture
level and the implementation level. We anticipate filing sub-issues that break down the tasks
that must be completed to achieve this goal.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org


Mime
View raw message