This patch adds an operator called repartition with more straightforward
semantics than the current `coalesce` operator. There are a few use cases
where this operator is useful:
1. If a user wants to increase the number of partitions in the RDD. This
is more common now with streaming. E.g. a user is ingesting data on one
node but they want to add more partitions to ensure parallelism of
subsequent operations across threads or the cluster.
Right now they have to call rdd.coalesce(numSplits, shuffle=true) - that's
super confusing.
2. If a user has input data where the number of partitions is not known. E.g.
> sc.textFile("some file").coalesce(50)....
This is both vague semantically (am I growing or shrinking this RDD) but also,
may not work correctly if the base RDD has fewer than 50 partitions.
The new operator forces shuffles every time, so it will always produce exactly
the number of new partitions. It also throws an exception rather than silently
not-working if a bad input is passed.
I am currently adding streaming tests (requires refactoring some of the test
suite to allow testing at partition granularity), so this is not ready for
merge yet. But feedback is welcome.
Add classmethod to SparkContext to set system properties.
Add a new classmethod to SparkContext to set system properties like is
possible in Scala/Java. Unlike the Java/Scala implementations, there's
no access to System until the JVM bridge is created. Since
SparkContext handles that, move the initialization of the JVM
connection to a separate classmethod that can safely be called
repeatedly as long as the same instance (or no instance) is provided.
Standalone Scheduler fault tolerance using ZooKeeper
This patch implements full distributed fault tolerance for standalone scheduler Masters.
There is only one master Leader at a time, which is actively serving scheduling
requests. If this Leader crashes, another master will eventually be elected, reconstruct
the state from the first Master, and continue serving scheduling requests.
Leader election is performed using the ZooKeeper leader election pattern. We try to minimize
the use of ZooKeeper and the assumptions about ZooKeeper's behavior, so there is a layer of
retries and session monitoring on top of the ZooKeeper client.
Master failover follows directly from the single-node Master recovery via the file
system (patch d5a96fe), save that the Master state is stored in ZooKeeper instead.
Configuration:
By default, no recovery mechanism is enabled (spark.deploy.recoveryMode = NONE).
By setting spark.deploy.recoveryMode to ZOOKEEPER and setting spark.deploy.zookeeper.url
to an appropriate ZooKeeper URL, ZooKeeper recovery mode is enabled.
By setting spark.deploy.recoveryMode to FILESYSTEM and setting spark.deploy.recoveryDirectory
to an appropriate directory accessible by the Master, we will keep the behavior of from d5a96fe.
Additionally, places where a Master could be specificied by a spark:// url can now take
comma-delimited lists to specify backup masters. Note that this is only used for registration
of NEW Workers and application Clients. Once a Worker or Client has registered with the
Master Leader, it is "in the system" and will never need to register again.