aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* Merge remote-tracking branch 'tgravescs/sparkYarnDistCache'Matei Zaharia2013-10-101-3/+14
|\ | | | | | | | | | | | | | | Closes #11 Conflicts: docs/running-on-yarn.md yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
| * Adding in the --addJars option to make SparkContext.addJar work on yarn and ↵tgravescs2013-10-031-3/+14
| | | | | | | | | | | | cleanup the classpaths
* | Merge pull request #19 from aarondav/master-zkMatei Zaharia2013-10-1030-169/+1673
|\ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
| * | Address Matei's comments on documentationAaron Davidson2013-10-103-6/+10
| | | | | | | | | | | | Updates to the documentation and changing some logError()s to logWarning()s.
| * | Revert change to spark-classAaron Davidson2013-10-081-2/+7
| | | | | | | | | | | | Also adds comment about how to configure for FaultToleranceTest.
| * | Change url format to spark://host1:port1,host2:port2Aaron Davidson2013-10-065-7/+14
| | | | | | | | | | | | | | | This replaces the format of spark://host1:port1,spark://host2:port2 and is more consistent with ZooKeeper's zk:// urls.
| * | Add end-to-end test for standalone scheduler fault toleranceAaron Davidson2013-10-052-1/+414
| | | | | | | | | | | | Docker files drawn mostly from Matt Masse. Some updates from Andre Schumacher.
| * | Address Matei's commentsAaron Davidson2013-10-0510-40/+41
| | |
| * | Fix race conditions during recoveryAaron Davidson2013-10-048-52/+122
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | One major change was the use of messages instead of raw functions as the parameter of Akka scheduled timers. Since messages are serialized, unlike raw functions, the behavior is easier to think about and doesn't cause race conditions when exceptions are thrown. Another change is to avoid using global pointers that might change without a lock.
| * | Add license noticesAaron Davidson2013-09-266-3/+86
| | |
| * | Standalone Scheduler fault tolerance using ZooKeeperAaron Davidson2013-09-2623-170/+708
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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 194ba4b8), 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 194ba4b8. 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. Forthcoming: Documentation, tests (! - only ad hoc testing has been performed so far) I do not intend for this commit to be merged until tests are added, but this patch should still be mostly reviewable until then.
| * | Standalone Scheduler fault recoveryAaron Davidson2013-09-2615-75/+458
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Implements a basic form of Standalone Scheduler fault recovery. In particular, this allows faults to be manually recovered from by means of restarting the Master process on the same machine. This is the majority of the code necessary for general fault tolerance, which will first elect a leader and then recover the Master state. In order to enable fault recovery, the Master will persist a small amount of state related to the registration of Workers and Applications to disk. If the Master is started and sees that this state is still around, it will enter Recovery mode, during which time it will not schedule any new Executors on Workers (but it does accept the registration of new Clients and Workers). At this point, the Master attempts to reconnect to all Workers and Client applications that were registered at the time of failure. After confirming either the existence or nonexistence of all such nodes (within a certain timeout), the Master will exit Recovery mode and resume normal scheduling.
* | | Merge branch 'master' into fast-mapMatei Zaharia2013-10-1012-47/+96
|\ \ \ | | | | | | | | | | | | | | | | Conflicts: core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
| * \ \ Merge pull request #49 from mateiz/kryo-fix-2Reynold Xin2013-10-092-3/+32
| |\ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | Fix Chill serialization of Range objects It used to write out each element one by one, creating very large objects.
| | * | | Fix Chill serialization of Range objects, which used to write out eachMatei Zaharia2013-10-092-3/+32
| | | | | | | | | | | | | | | | | | | | | | | | | element, and register user and Spark classes before Chill's serializers to let them override Chill's behavior in general.
| * | | | Merge pull request #50 from kayousterhout/SPARK-908Reynold Xin2013-10-094-17/+37
| |\ \ \ \ | | |/ / / | |/| | | | | | | | Fix race condition in SparkListenerSuite (fixes SPARK-908).
| | * | | Style fixesKay Ousterhout2013-10-091-6/+13
| | | | |
| | * | | Fixed comment to use javadoc styleKay Ousterhout2013-10-091-5/+5
| | | | |
| | * | | Fix race condition in SparkListenerSuite (fixes SPARK-908).Kay Ousterhout2013-10-094-12/+25
| | | | |
| * | | | Use standard abbreviations in metrics labelsPatrick Wendell2013-10-092-6/+6
| | | | |
| * | | | Merge pull request #22 from GraceH/metrics-namingPatrick Wendell2013-10-096-27/+27
| |\ \ \ \ | | |/ / / | |/| | | | | | | | | | | | | | | | | | | | | | | SPARK-900 Use coarser grained naming for metrics see SPARK-900 Use coarser grained naming for metrics. Now the new metric name is formatted as {XXX.YYY.ZZZ.COUNTER_UNIT}, XXX.YYY.ZZZ represents the group name, which can group several metrics under the same Ganglia view.
| | * | | remove those futile suffixes like number/countGrace Huang2013-10-095-14/+14
| | | | |
| | * | | create metrics name manually.Grace Huang2013-10-086-27/+27
| | | | |
| | * | | Revert "SPARK-900 Use coarser grained naming for metrics"Grace Huang2013-10-087-96/+28
| | | | | | | | | | | | | | | | | | | | This reverts commit 4b68be5f3c0a251453c184b233b3ca490812dafd.
| | * | | Revert "remedy the line-wrap while exceeding 100 chars"Grace Huang2013-10-086-114/+91
| | | | | | | | | | | | | | | | | | | | This reverts commit 892fb8ffa85016a63d7d00dd6f1abc58ccf034a2.
| | * | | remedy the line-wrap while exceeding 100 charsGrace Huang2013-09-306-91/+114
| | | | |
| | * | | SPARK-900 Use coarser grained naming for metricsGrace Huang2013-09-277-28/+96
| | |/ /
* | | | Create fewer function objects in uses of AppendOnlyMap.changeValueMatei Zaharia2013-10-082-20/+20
| | | |
* | | | Address some comments on code clarityMatei Zaharia2013-10-081-9/+11
| | | |
* | | | Moved files that were in the wrong directory after package renameMatei Zaharia2013-10-082-0/+0
| | | |
* | | | Fix some review commentsMatei Zaharia2013-10-084-38/+33
| | | |
* | | | Added a fast and low-memory append-only map implementation for cogroupMatei Zaharia2013-10-085-29/+411
|/ / / | | | | | | | | | and parallel reduce operations
* | | Merge pull request #43 from mateiz/kryo-fixReynold Xin2013-10-081-4/+4
|\ \ \ | | | | | | | | | | | | | | | | | | | | Don't allocate Kryo buffers unless needed I noticed that the Kryo serializer could be slower than the Java one by 2-3x on small shuffles because it spend a lot of time initializing Kryo Input and Output objects. This is because our default buffer size for them is very large. Since the serializer is often used on streams, I made the initialization lazy for that, and used a smaller buffer (auto-managed by Kryo) for input.
| * | | Don't allocate Kryo buffers unless neededMatei Zaharia2013-10-071-4/+4
| | | |
* | | | Responses to reviewPatrick Wendell2013-10-071-4/+4
| | | |
* | | | Fix inconsistent and incorrect log messages in shuffle read pathPatrick Wendell2013-10-072-7/+16
| | | |
* | | | Merge pull request #31 from sundeepn/branch-0.8Reynold Xin2013-10-071-1/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Resolving package conflicts with hadoop 0.23.9 Hadoop 0.23.9 is having a package conflict with easymock's dependencies. (cherry picked from commit 023e3fdf008b3194a36985a07923df9aaf64e520) Signed-off-by: Reynold Xin <rxin@apache.org>
* | | | Merging build changes in from 0.8Patrick Wendell2013-10-052-9/+12
|/ / /
* | | Merge pull request #20 from harveyfeng/hadoop-config-cacheMatei Zaharia2013-10-054-34/+161
|\ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Allow users to pass broadcasted Configurations and cache InputFormats across Hadoop file reads. Note: originally from https://github.com/mesos/spark/pull/942 Currently motivated by Shark queries on Hive-partitioned tables, where there's a JobConf broadcast for every Hive-partition (i.e., every subdirectory read). The only thing different about those JobConfs is the input path - the Hadoop Configuration that the JobConfs are constructed from remain the same. This PR only modifies the old Hadoop API RDDs, but similar additions to the new API might reduce computation latencies a little bit for high-frequency FileInputDStreams (which only uses the new API right now). As a small bonus, added InputFormats caching, to avoid reflection calls for every RDD#compute(). Few other notes: Added a general soft-reference hashmap in SparkHadoopUtil because I wanted to avoid adding another class to SparkEnv. SparkContext default hadoopConfiguration isn't cached. There's no equals() method for Configuration, so there isn't a good way to determine when configuration properties have changed.
| * | | Some comments regarding JobConf and InputFormat caching for HadoopRDDs.Harvey Feng2013-10-052-4/+25
| | | |
| * | | Make HadoopRDD object Spark private.Harvey Feng2013-10-051-3/+3
| | | |
| * | | Fix API changes; lines > 100 chars.Harvey Feng2013-10-052-15/+18
| | | |
| * | | Merge HadoopDatasetRDD into HadoopRDD.Harvey Feng2013-09-292-27/+40
| | | |
| * | | Merge remote-tracking branch 'oldsparkme/hadoopRDD-broadcast-change' into ↵Harvey Feng2013-09-264-30/+120
| |\ \ \ | | |/ / | |/| | | | | | hadoop-config-cache
| | * | Move Configuration broadcasts to SparkContext.Harvey2013-09-222-8/+6
| | | |
| | * | Add a cache for HadoopRDD metadata needed during computation.Harvey2013-09-224-37/+79
| | | | | | | | | | | | Currently, the cache is in SparkHadoopUtils, since it's conveniently a member of the SparkEnv.
| | * | Split HadoopRDD into one for general Hadoop datasets and one tailored to ↵Harvey2013-09-212-21/+71
| | |/ | | | | | | | | | | | | Hadoop files, which is a common case. This is the first step to avoiding unnecessary Configuration broadcasts per HadoopRDD instantiation.
* | | Merge pull request #27 from davidmccauley/masterMatei Zaharia2013-10-051-1/+2
|\ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | SPARK-920/921 - JSON endpoint updates 920 - Removal of duplicate scheme part of Spark URI, it was appearing as spark://spark//host:port in the JSON field. JSON now delivered as: url:spark://127.0.0.1:7077 921 - Adding the URL of the Main Application UI will allow custom interfaces (that use the JSON output) to redirect from the standalone UI.
| * | | SPARK-921 - Add Application UI URL to ApplicationInfo Json outputDavid McCauley2013-10-021-0/+1
| | | |
| * | | SPARK-920 - JSON endpoint URI scheme part (spark://) duplicatedDavid McCauley2013-10-021-1/+1
| |/ /