Commit graph

2333 commits

Author SHA1 Message Date
Kay Ousterhout 3f7e9b265c Fixed comment to use javadoc style 2013-10-09 15:23:04 -07:00
Kay Ousterhout a34a4e8174 Fix race condition in SparkListenerSuite (fixes SPARK-908). 2013-10-09 15:07:53 -07:00
Patrick Wendell bd3bcc5f8e Use standard abbreviations in metrics labels 2013-10-09 11:16:24 -07:00
Patrick Wendell 19d445d37c Merge pull request #22 from GraceH/metrics-naming
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.
2013-10-09 11:08:34 -07:00
Matei Zaharia 12d593129d Create fewer function objects in uses of AppendOnlyMap.changeValue 2013-10-08 23:16:51 -07:00
Matei Zaharia 0b35051f19 Address some comments on code clarity 2013-10-08 23:16:17 -07:00
Matei Zaharia 4acbc5afdd Moved files that were in the wrong directory after package rename 2013-10-08 23:16:17 -07:00
Matei Zaharia 0e40cfabf8 Fix some review comments 2013-10-08 23:16:16 -07:00
Matei Zaharia b535db7d89 Added a fast and low-memory append-only map implementation for cogroup
and parallel reduce operations
2013-10-08 23:14:38 -07:00
Reynold Xin e67d5b962a Merge pull request #43 from mateiz/kryo-fix
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.
2013-10-08 22:57:38 -07:00
Grace Huang f7628e4033 remove those futile suffixes like number/count 2013-10-09 08:36:41 +08:00
Aaron Davidson 749233b869 Revert change to spark-class
Also adds comment about how to configure for FaultToleranceTest.
2013-10-08 11:41:52 -07:00
Grace Huang 22bed59d2d create metrics name manually. 2013-10-08 18:01:11 +08:00
Grace Huang 188abbf8f1 Revert "SPARK-900 Use coarser grained naming for metrics"
This reverts commit 4b68be5f3c.
2013-10-08 17:45:14 +08:00
Grace Huang a2af6b543a Revert "remedy the line-wrap while exceeding 100 chars"
This reverts commit 892fb8ffa8.
2013-10-08 17:44:56 +08:00
Patrick Wendell 9e9e9e1b42 Making the timing block more narrow for the sync 2013-10-07 21:28:12 -07:00
Patrick Wendell 8b377718b8 Responses to review 2013-10-07 20:03:35 -07:00
Matei Zaharia a8725bf8f8 Don't allocate Kryo buffers unless needed 2013-10-07 19:16:35 -07:00
Patrick Wendell 391133f66a Fix inconsistent and incorrect log messages in shuffle read path 2013-10-07 17:24:18 -07:00
Patrick Wendell b08306c5cf Minor cleanup 2013-10-07 16:30:25 -07:00
Patrick Wendell 524d01ea31 Perf benchmark 2013-10-07 15:15:42 -07:00
Patrick Wendell d15acd6457 Trying new approach with writes 2013-10-07 15:15:42 -07:00
Patrick Wendell a224c8c9b8 Adding option to force sync to the filesystem 2013-10-07 15:15:42 -07:00
Patrick Wendell 3478ca6762 Track and report write throughput for shuffle tasks. 2013-10-07 15:15:41 -07:00
Reynold Xin 213b70a2db Merge pull request #31 from sundeepn/branch-0.8
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 023e3fdf00)
Signed-off-by: Reynold Xin <rxin@apache.org>
2013-10-07 10:54:22 -07:00
Kay Ousterhout fdc52b2f8b Added back fully qualified class name 2013-10-06 18:45:43 -07:00
Aaron Davidson 718e8c2052 Change url format to spark://host1:port1,host2:port2
This replaces the format of spark://host1:port1,spark://host2:port2 and is more
consistent with ZooKeeper's zk:// urls.
2013-10-06 00:02:08 -07:00
Aaron Davidson e1190229e1 Add end-to-end test for standalone scheduler fault tolerance
Docker files drawn mostly from Matt Masse. Some updates from Andre Schumacher.
2013-10-05 23:20:31 -07:00
Patrick Wendell aa9fb84994 Merging build changes in from 0.8 2013-10-05 22:07:00 -07:00
Matei Zaharia 4a25b116d4 Merge pull request #20 from harveyfeng/hadoop-config-cache
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.
2013-10-05 19:28:55 -07:00
Harvey Feng 6a2bbec5e3 Some comments regarding JobConf and InputFormat caching for HadoopRDDs. 2013-10-05 17:53:58 -07:00
Harvey Feng 96929f28bb Make HadoopRDD object Spark private. 2013-10-05 17:14:19 -07:00
Harvey Feng b5e93c1227 Fix API changes; lines > 100 chars. 2013-10-05 16:57:08 -07:00
Aaron Davidson 0f070279e7 Address Matei's comments 2013-10-05 15:15:29 -07:00
Matei Zaharia 100222b048 Merge pull request #27 from davidmccauley/master
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.
2013-10-05 13:38:59 -07:00
Mridul Muralidharan b5025d90bb - Allow for finer control of cleaner
- Address review comments, move to incubator spark
- Also includes a change to speculation - including preventing exceptions in rare cases.
2013-10-06 00:35:51 +05:30
Aaron Davidson db6f154940 Fix race conditions during recovery
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.
2013-10-04 19:54:33 -07:00
Kay Ousterhout 7b5ae23a37 Renamed StandaloneX to CoarseGrainedX.
The previous names were confusing because the components weren't just
used in Standalone mode -- in fact, the scheduler used for Standalone
mode is called SparkDeploySchedulerBackend. So, the previous names
were misleading.
2013-10-04 13:56:43 -07:00
Andre Schumacher c84946fe21 Fixing SPARK-602: PythonPartitioner
Currently PythonPartitioner determines partition ID by hashing a
byte-array representation of PySpark's key. This PR lets
PythonPartitioner use the actual partition ID, which is required e.g.
for sorting via PySpark.
2013-10-04 11:56:47 -07:00
Reynold Xin d29e8035a0 Added countAsync and various unit tests for async actions. 2013-10-03 15:13:44 -07:00
tgravescs 0fff4ee852 Adding in the --addJars option to make SparkContext.addJar work on yarn and cleanup
the classpaths
2013-10-03 11:52:16 -05:00
Reynold Xin 802bfb870d - Created AsyncRDDActions.
- Make FutureJob a Scala Future instead of Java Future.
2013-10-03 01:22:28 -07:00
Reynold Xin e8e917f209 Merge branch 'master' into kill
Conflicts:
	core/src/main/scala/org/apache/spark/TaskEndReason.scala
	core/src/main/scala/org/apache/spark/executor/Executor.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
2013-10-02 23:01:34 -07:00
Reynold Xin 1c48ba0d9f Merge remote-tracking branch 'origin' into kill
Conflicts:
	core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
2013-10-02 16:40:44 -07:00
David McCauley 1577b373a9 SPARK-921 - Add Application UI URL to ApplicationInfo Json output 2013-10-02 15:03:41 +01:00
David McCauley 351da54676 SPARK-920 - JSON endpoint URI scheme part (spark://) duplicated 2013-10-02 13:23:38 +01:00
Kay Ousterhout 0dcad2edcb Added additional unit test for repeated task failures 2013-09-30 23:26:15 -07:00
Kay Ousterhout dea4677c88 Fixed compilation errors and broken test. 2013-09-30 22:07:01 -07:00
Kay Ousterhout 8deda427bc Merge remote-tracking branch 'upstream/master' into results_through-bm
Conflicts:
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
2013-09-30 10:16:58 -07:00
Kay Ousterhout 58b764b7c6 Addressed Matei's code review comments 2013-09-30 10:11:59 -07:00
Grace Huang 892fb8ffa8 remedy the line-wrap while exceeding 100 chars 2013-09-30 20:12:55 +08:00
Harvey Feng 7d06bdde1d Merge HadoopDatasetRDD into HadoopRDD. 2013-09-29 20:08:03 -07:00
Grace Huang 4b68be5f3c SPARK-900 Use coarser grained naming for metrics 2013-09-27 14:47:38 +08:00
Harvey Feng 417085716a Merge remote-tracking branch 'oldsparkme/hadoopRDD-broadcast-change' into hadoop-config-cache 2013-09-26 15:49:42 -07:00
Aaron Davidson 42d72308fb Add license notices 2013-09-26 15:45:20 -07:00
Aaron Davidson f549ea33d3 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 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.
2013-09-26 15:04:23 -07:00
Aaron Davidson d5a96feccb Standalone Scheduler fault recovery
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.
2013-09-26 14:59:35 -07:00
Reynold Xin 70a0b993d4 Merge pull request #14 from kayousterhout/untangle_scheduler
Improved organization of scheduling packages.

This commit does not change any code -- only file organization.
Please let me know if there was some masterminded strategy behind
the existing organization that I failed to understand!

There are two components of this change:
(1) Moving files out of the cluster package, and down
a level to the scheduling package. These files are all used by
the local scheduler in addition to the cluster scheduler(s), so
should not be in the cluster package. As a result of this change,
none of the files in the local package reference files in the
cluster package.

(2) Moving the mesos package to within the cluster package.
The mesos scheduling code is for a cluster, and represents a
specific case of cluster scheduling (the Mesos-related classes
often subclass cluster scheduling classes). Thus, the most logical
place for it seems to be within the cluster package.

The one thing about the scheduling code that seems a little funny to me
is the naming of the SchedulerBackends.  The StandaloneSchedulerBackend
is not just for Standalone mode, but instead is used by Mesos coarse grained
mode and Yarn, and the backend that *is* just for Standalone mode is instead called SparkDeploySchedulerBackend. I didn't change this because I wasn't sure if there
was a reason for this naming that I'm just not aware of.
2013-09-26 14:11:54 -07:00
Reynold Xin c514cd1587 Merge pull request #930 from holdenk/master
Add mapPartitionsWithIndex
2013-09-26 13:48:20 -07:00
Reynold Xin 560ee5c9bb Merge pull request #7 from wannabeast/memorystore-fixes
some minor fixes to MemoryStore

This is a repeat of #5, moved to its own branch in my repo.

This makes all updates to   on ; it skips on synchronizing the reads where it can get away with it.
2013-09-26 11:27:34 -07:00
Patrick Wendell 6566a19b38 Merge pull request #9 from rxin/limit
Smarter take/limit implementation.
2013-09-26 08:01:04 -07:00
Kay Ousterhout d85fe41b2b Improved organization of scheduling packages.
This commit does not change any code -- only file organization.

There are two components of this change:
(1) Moving files out of the cluster package, and down
a level to the scheduling package. These files are all used by
the local scheduler in addition to the cluster scheduler(s), so
should not be in the cluster package. As a result of this change,
none of the files in the local package reference files in the
cluster package.

(2) Moving the mesos package to within the cluster package.
The mesos scheduling code is for a cluster, and represents a
specific case of cluster scheduling (the Mesos-related classes
often subclass cluster scheduling classes). Thus, the most logical
place for it is within the cluster package.
2013-09-25 12:45:46 -07:00
Patrick Wendell 6079721fa1 Update build version in master 2013-09-24 11:41:51 -07:00
Holden Karau 0cef683553 Fix formatting :) 2013-09-23 19:39:42 -07:00
Reynold Xin ff540a015b Merge branch 'master' of github.com:markhamstra/incubator-spark 2013-09-23 11:55:02 -07:00
Kay Ousterhout c75eb14fe5 Send Task results through the block manager when larger than Akka frame size.
This change requires adding an extra failure mode: tasks can complete
successfully, but the result gets lost or flushed from the block manager
before it's been fetched.
2013-09-22 21:20:48 -07:00
Holden Karau 7fe0b0ff56 Switch indent from 2 to 4 spaces 2013-09-22 19:44:51 -07:00
Harvey ef34cfb26c Move Configuration broadcasts to SparkContext. 2013-09-22 14:43:58 -07:00
Harvey a6eeb5ffd5 Add a cache for HadoopRDD metadata needed during computation.
Currently, the cache is in SparkHadoopUtils, since it's conveniently a member of the SparkEnv.
2013-09-22 03:09:17 -07:00
jerryshao 77e9da1f34 Change Exception to NoSuchElementException and minor style fix 2013-09-22 16:50:08 +08:00
jerryshao 85024acd2e Remove infix style and others 2013-09-22 14:20:55 +08:00
jerryshao 5850f599dd Refactor FairSchedulableBuilder:
1. Configuration can be read from classpath if not set explicitly.
2. Add missing close handler.
2013-09-22 14:20:55 +08:00
Reynold Xin a2ea069a5f Merge pull request #937 from jerryshao/localProperties-fix
Fix PR926 local properties issues in Spark Streaming like scenarios
2013-09-21 23:04:42 -07:00
Harvey be0fc7246f Split HadoopRDD into one for general Hadoop datasets and one tailored to Hadoop files, which is a common case.
This is the first step to avoiding unnecessary Configuration broadcasts per HadoopRDD instantiation.
2013-09-21 21:14:14 -07:00
jerryshao aa0c29f747 Add barrier for local properties unit test and fix some styles 2013-09-22 09:53:11 +08:00
Reynold Xin 42571d30d0 Smarter take/limit implementation. 2013-09-20 17:09:53 -07:00
Reynold Xin 1d87616b61 Made output of CoGroup and aggregations interruptible. 2013-09-19 23:31:36 -07:00
Mike 9524b943a4 Synchronize on "entries" the remaining update to "currentMemory".
Make "currentMemory" @volatile, so that it's reads in ensureFreeSpace() are atomic and up-to-date--i.e., currentMemory can't increase while putLock is held (though it could decrease, which would only help ensureFreeSpace()).
2013-09-19 23:31:35 -07:00
Ankur Dave 026dba6aba After unit tests, clear port properties unconditionally
In MapOutputTrackerSuite, the "remote fetch" test sets spark.driver.port
and spark.hostPort, assuming that they will be cleared by
LocalSparkContext. However, the test never sets sc, so it remains null,
causing LocalSparkContext to skip clearing these properties. Subsequent
tests therefore fail with java.net.BindException: "Address already in
use".

This commit makes LocalSparkContext clear the properties even if sc is
null.
2013-09-19 22:05:23 -07:00
Reynold Xin c5e40954eb Wrap around cached data to InterruptibleIterator. 2013-09-19 18:44:38 -07:00
Reynold Xin c68e72be59 Added comment to InterruptibleIterator. 2013-09-19 18:40:06 -07:00
Reynold Xin 70953810b4 Added task killing iterator to RDDs that take inputs. 2013-09-19 18:33:16 -07:00
Reynold Xin f19984dafe More logging changes (task killing for local cluster doesn't work yet). 2013-09-19 18:14:51 -07:00
Reynold Xin 85a0dffe0f Made task killing work for standalone cluster schedulers. 2013-09-19 16:41:29 -07:00
Reynold Xin 9f8190c17d Fixed a bug for zero partition in JobWaiter. 2013-09-18 22:42:35 -07:00
Reynold Xin 9332246bd0 Added a hack to kill all active jobs in SparkContext. 2013-09-18 04:38:24 -07:00
Reynold Xin bf515688e7 Allow SparkContext.submitJob to submit a job for only a subset of the partitions. 2013-09-18 04:16:18 -07:00
jerryshao ffa5f8e11d Fix issue when local properties pass from parent to child thread 2013-09-18 17:33:24 +08:00
Reynold Xin 37d8f37a8e Added a submitJob interface that returns a Future of the result. 2013-09-17 21:13:59 -07:00
Reynold Xin 1cb42e6b2d Properly handle job failure when the job gets killed. 2013-09-16 22:10:45 -07:00
Reynold Xin cbc48be13b Initial commit for job killing. 2013-09-16 18:54:06 -07:00
Holden Karau bfcddf4700 Make mapPartitionsWithIndex work with JavaRDD's 2013-09-14 15:53:42 -07:00
Holden Karau 74f710f6cd Start of working on SPARK-615 2013-09-11 22:35:58 -07:00
Mike d34672f668 Set currentMemory to 0 in clear().
Remove unnecessary entries.get() call.
2013-09-11 18:01:19 -07:00
Kay Ousterhout 93c4253275 Changed localProperties to use ThreadLocal (not DynamicVariable).
The fact that DynamicVariable uses an InheritableThreadLocal
can cause problems where the properties end up being shared
across threads in certain circumstances.
2013-09-11 13:01:39 -07:00
Patrick Wendell 91a59e6b10 Merge pull request #919 from mateiz/jets3t
Add explicit jets3t dependency, which is excluded in hadoop-client
2013-09-11 10:21:48 -07:00
Patrick Wendell b9128d34bf Merge pull request #922 from pwendell/port-change
Change default port number from 3030 to 4030.
2013-09-11 10:03:06 -07:00
Patrick Wendell bddf135670 Change port from 3030 to 4040 2013-09-11 10:01:38 -07:00
David McCauley 5dd875c5b5 SPARK-894 - Not all WebUI fields delivered VIA JSON 2013-09-11 10:46:37 +01:00
Mike 293c758cc0 Remove MemoryStore$Entry.dropPending, unused as of 42e0a68082. 2013-09-10 00:24:35 -07:00
Matei Zaharia f117dc6d0d Add explicit jets3t dependency, which is excluded in hadoop-client 2013-09-10 06:39:25 +00:00
Matei Zaharia c81377b9ed Merge pull request #915 from ooyala/master
Get rid of / improve ugly NPE when Utils.deleteRecursively() fails
2013-09-09 20:16:19 -07:00
Evan Chan fdb8b0eec3 Style fix: put body of if within curly braces 2013-09-09 14:29:32 -07:00
Matei Zaharia a85758c200 Merge pull request #907 from stephenh/document_coalesce_shuffle
Add better docs for coalesce.
2013-09-09 13:45:40 -07:00
Evan Chan 27726079e4 Print out more friendly error if listFiles() fails
listFiles() could return null if the I/O fails, and this currently results in an ugly NPE which is hard to diagnose.
2013-09-09 12:58:12 -07:00
Y.CORP.YAHOO.COM\tgraves 2186d93285 Add metrics-ganglia to core pom file 2013-09-09 12:37:33 -05:00
Stephen Haberman 59003d387d Use a set since shuffle could change order. 2013-09-09 11:45:03 -05:00
Stephen Haberman 6471bfec73 Reword 'evenly distributed' to 'distributed with a hash partitioner. 2013-09-09 11:44:15 -05:00
Matei Zaharia bf984e2745 Merge pull request #890 from mridulm/master
Fix hash bug
2013-09-08 23:50:24 -07:00
Reynold Xin e9d4f44a7a Merge pull request #909 from mateiz/exec-id-fix
Fix an instance where full standalone mode executor IDs were passed to
2013-09-08 23:36:48 -07:00
Matei Zaharia 7d3204b056 Merge pull request #905 from mateiz/docs2
Job scheduling and cluster mode docs
2013-09-08 21:39:12 -07:00
Patrick Wendell f68848d95d Merge pull request #906 from pwendell/ganglia-sink
Clean-up of Metrics Code/Docs and Add Ganglia Sink
2013-09-08 18:32:16 -07:00
Matei Zaharia f9b7f58de2 Fix an instance where full standalone mode executor IDs were passed to
StandaloneSchedulerBackend instead of the smaller IDs used within Spark
(that lack the application name).

This was reported by ClearStory in
https://github.com/clearstorydata/spark/pull/9.

Also fixed some messages that said slave instead of executor.
2013-09-08 18:27:50 -07:00
Matei Zaharia 170b3869ee Fix unit test failure due to changed default 2013-09-08 17:51:27 -07:00
Patrick Wendell b4e382c210 Adding sc name in metrics source 2013-09-08 16:06:49 -07:00
Patrick Wendell c190b48bf5 Adding more docs and some code cleanup 2013-09-08 13:46:28 -07:00
Stephen Haberman df5fd35273 Add better docs for coalesce.
Include the useful tip that if shuffle=true, coalesce can actually
increase the number of partitions.

This makes coalesce more like a generic `RDD.repartition` operation.

(Ideally this `RDD.repartition` could automatically choose either a coalesce or
a shuffle if numPartitions was either less than or greater than, respectively,
the current number of partitions.)
2013-09-08 15:39:04 -05:00
Matei Zaharia 04cfb3aa9d Merge pull request #898 from ilikerps/660
SPARK-660: Add StorageLevel support in Python
2013-09-08 10:33:20 -07:00
Patrick Wendell 8de8ee5d3c Ganglia sink 2013-09-08 10:08:18 -07:00
Matei Zaharia 651a96adf7 More fair scheduler docs and property names.
Also changed uses of "job" terminology to "application" when they
referred to an entire Spark program, to avoid confusion.
2013-09-08 00:29:11 -07:00
Matei Zaharia 98fb69822c Work in progress:
- Add job scheduling docs
- Rename some fair scheduler properties
- Organize intro page better
- Link to Apache wiki for "contributing to Spark"
2013-09-08 00:29:11 -07:00
Aaron Davidson c1cc8c4da2 Export StorageLevel and refactor 2013-09-07 14:41:31 -07:00
Aaron Davidson 8001687af5 Remove reflection, hard-code StorageLevels
The sc.StorageLevel -> StorageLevel pathway is a bit janky, but otherwise
the shell would have to call a private method of SparkContext. Having
StorageLevel available in sc also doesn't seem like the end of the world.
There may be a better solution, though.

As for creating the StorageLevel object itself, this seems to be the best
way in Python 2 for creating singleton, enum-like objects:
http://stackoverflow.com/questions/36932/how-can-i-represent-an-enum-in-python
2013-09-07 09:34:07 -07:00
Reynold Xin 210eae26f4 Fixed the bug that ResultTask was not properly deserializing outputId. 2013-09-07 21:59:47 +08:00
Aaron Davidson b8a0b6ea5e Memoize StorageLevels read from JVM 2013-09-06 15:36:04 -07:00
Reynold Xin 1e15feb5a3 Hot fix to resolve the compilation error caused by SPARK-821. 2013-09-06 22:44:05 +08:00
Patrick Wendell ddcb9d310a Merge pull request #895 from ilikerps/821
SPARK-821: Don't cache results when action run locally on driver
2013-09-05 23:54:09 -07:00
Aaron Davidson a63d4c7dc2 SPARK-660: Add StorageLevel support in Python
It uses reflection... I am not proud of that fact, but it at least ensures
compatibility (sans refactoring of the StorageLevel stuff).
2013-09-05 23:36:27 -07:00
Aaron Davidson 3a04e76c89 Reynold's second round of comments 2013-09-05 21:43:26 -07:00
Matei Zaharia 699c331f2f Merge pull request #891 from xiajunluan/SPARK-864
[SPARK-864]DAGScheduler Exception if we delete Worker and StandaloneExecutorBackend then add Worker
2013-09-05 20:21:53 -07:00
Aaron Davidson 4f2236a1c5 Add unit test and address comments 2013-09-05 18:06:30 -07:00
Aaron Davidson 1418d18af4 SPARK-821: Don't cache results when action run locally on driver
Caching the results of local actions (e.g., rdd.first()) causes the driver to
store entire partitions in its own memory, which may be highly constrained.
This patch simply makes the CacheManager avoid caching the result of all locally-run computations.
2013-09-05 15:34:42 -07:00
Andrew xia 7c15e3c5de Fix bug SPARK-864 2013-09-05 15:56:11 +08:00
Patrick Wendell 5c7494d7c1 Merge pull request #893 from ilikerps/master
SPARK-884: Add unit test to validate Spark JSON output
2013-09-04 22:47:03 -07:00
Aaron Davidson 714e7f9e32 Fix line over 100 chars 2013-09-04 22:40:08 -07:00
Aaron Davidson 37db141aef Address Patrick's comments 2013-09-04 21:34:20 -07:00
Aaron Davidson 9e6f2b6822 SPARK-884: Add unit test to validate Spark JSON output
This unit test simply validates that the outputs of
the JsonProtocol methods are syntactically valid JSON.
2013-09-04 15:26:46 -07:00
Mridul Muralidharan 1e2474b814 Address review comments - rename toHash to nonNegativeHash 2013-09-04 07:46:46 +05:30
Mridul Muralidharan b3a82b7df3 Fix hash bug - caused failure after 35k stages, sigh 2013-09-04 07:02:25 +05:30
Mark Hamstra c9bc8af3d1 Removed repetative import; fixes hidden definition compiler warning. 2013-09-03 15:25:20 -07:00
Patrick Wendell c592a3c9b9 Minor spacing fix 2013-09-03 14:39:11 -07:00
Patrick Wendell 19f70273d2 Merge pull request #878 from tgravescs/yarnUILink
Link the Spark UI up to the Yarn UI
2013-09-03 14:29:10 -07:00
Matei Zaharia 68df2464d1 Merge pull request #889 from alig/master
Return the port the WebUI is bound to (useful if port 0 was used)
2013-09-03 13:01:17 -07:00
Y.CORP.YAHOO.COM\tgraves 41c1b5b9a0 Update based on review comments. Change function to prependBaseUri and fix formatting. 2013-09-03 14:46:51 -05:00
Y.CORP.YAHOO.COM\tgraves c8cc276110 Review comment changes and update to org.apache packaging 2013-09-03 10:50:21 -05:00
Y.CORP.YAHOO.COM\tgraves 547fc4a412 Merge remote-tracking branch 'mesos/master' into yarnUILink
Conflicts:
	core/src/main/scala/org/apache/spark/ui/UIUtils.scala
	core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
	core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
	docs/running-on-yarn.md
2013-09-03 08:36:59 -05:00
Ali Ghodsi b25918d841 Merge branch 'master' of https://github.com/alig/spark
Conflicts:
	core/src/main/scala/org/apache/spark/deploy/master/Master.scala
2013-09-03 00:56:12 -07:00
Ali Ghodsi bd0788505f Using configured akka timeouts 2013-09-03 00:50:35 -07:00
Ali Ghodsi cbfef9b3ff Sort order of imports to match project guidelines 2013-09-02 19:33:55 -07:00
Ali Ghodsi 36d8fca2cc Reynold's comment fixed 2013-09-02 19:31:09 -07:00
Ali Ghodsi e452bd6d77 Brushing the code up slightly 2013-09-02 19:04:08 -07:00
Ali Ghodsi cf7b115496 Enabling getting the actual WEBUI port 2013-09-02 18:21:21 -07:00
Matei Zaharia 12b2f1f9c9 Add missing license headers found with RAT 2013-09-02 12:23:03 -07:00
Matei Zaharia 246bf67f58 Fix test 2013-09-02 10:57:34 -07:00
Matei Zaharia 9329a7d4cd Fix spark.io.compression.codec and change default codec to LZF 2013-09-02 10:15:22 -07:00
Matei Zaharia 6550e5e60c Allow PySpark to launch worker.py directly on Windows 2013-09-01 18:06:15 -07:00
Matei Zaharia 3db404a43a Run script fixes for Windows after package & assembly change 2013-09-01 23:45:57 +00:00
Matei Zaharia 0a8cc30921 Move some classes to more appropriate packages:
* RDD, *RDDFunctions -> org.apache.spark.rdd
* Utils, ClosureCleaner, SizeEstimator -> org.apache.spark.util
* JavaSerializer, KryoSerializer -> org.apache.spark.serializer
2013-09-01 14:13:16 -07:00
Matei Zaharia 5701eb92c7 Fix some URLs 2013-09-01 14:13:16 -07:00
Matei Zaharia 12495ec63a Remove shutdown hook to stop jetty; this is unnecessary for releasing
ports and creates noisy log messages
2013-09-01 14:13:15 -07:00
Matei Zaharia 46eecd110a Initial work to rename package to org.apache.spark 2013-09-01 14:13:13 -07:00
Matei Zaharia a30fac16ca Merge pull request #883 from alig/master
Don't require the spark home environment variable to be set for standalone mode (change needed by SIMR)
2013-09-01 12:27:50 -07:00
Matei Zaharia e34bc3a8ee Small tweak 2013-08-31 17:47:15 -07:00
Matei Zaharia 2ee6a7e32a Print output from spark-daemon only when it fails to launch 2013-08-31 17:31:07 -07:00
Ali Ghodsi 250bddc255 Don't require spark home to be set for standalone mode 2013-08-31 17:29:05 -07:00
Matei Zaharia 25ac50668b Various web UI improvements:
- Use "fluid" layout that can expand to wide browser windows, instead of
  the old one's limit of 1200 px
- Remove unnecessary <hr> elements
- Switch back to Bootstrap's default theme and tweak progress bar colors
- Make headers more consistent between deploy and app UIs
- Replace some inline CSS with stylesheets
2013-08-31 16:55:40 -07:00
Y.CORP.YAHOO.COM\tgraves 96452eea56 fix up minor things 2013-08-30 16:04:31 -05:00
Y.CORP.YAHOO.COM\tgraves bac46266a9 Link the Spark UI to the Yarn UI 2013-08-30 15:55:32 -05:00
Mikhail Bautin 35090958b3 Also add getConf to NewHadoopRDD 2013-08-30 11:03:57 -07:00
Mikhail Bautin 5e30172f70 Make HadoopRDD's configuration accessible 2013-08-30 11:01:06 -07:00
Matei Zaharia ca71620950 Merge pull request #857 from mateiz/assembly
Change build and run instructions to use assemblies
2013-08-29 21:51:14 -07:00
Matei Zaharia 666d93c294 Update Maven build to create assemblies expected by new scripts
This includes the following changes:
- The "assembly" package now builds in Maven by default, and creates an
  assembly containing both hadoop-client and Spark, unlike the old
  BigTop distribution assembly that skipped hadoop-client
- There is now a bigtop-dist package to build the old BigTop assembly
- The repl-bin package is no longer built by default since the scripts
  don't reply on it; instead it can be enabled with -Prepl-bin
- Py4J is now included in the assembly/lib folder as a local Maven repo,
  so that the Maven package can link to it
- run-example now adds the original Spark classpath as well because the
  Maven examples assembly lists spark-core and such as provided
- The various Maven projects add a spark-yarn dependency correctly
2013-08-29 21:19:06 -07:00
Matei Zaharia aab345c463 Fix finding of assembly JAR, as well as some pointers to ./run 2013-08-29 21:19:06 -07:00
Matei Zaharia ab0e625d9e Fix PySpark for assembly run and include it in dist 2013-08-29 21:19:06 -07:00
Matei Zaharia 53cd50c069 Change build and run instructions to use assemblies
This commit makes Spark invocation saner by using an assembly JAR to
find all of Spark's dependencies instead of adding all the JARs in
lib_managed. It also packages the examples into an assembly and uses
that as SPARK_EXAMPLES_JAR. Finally, it replaces the old "run" script
with two better-named scripts: "run-examples" for examples, and
"spark-class" for Spark internal classes (e.g. REPL, master, etc). This
is also designed to minimize the confusion people have in trying to use
"run" to run their own classes; it's not meant to do that, but now at
least if they look at it, they can modify run-examples to do a decent
job for them.

As part of this, Bagel's examples are also now properly moved to the
examples package instead of bagel.
2013-08-29 21:19:04 -07:00
jerryshao f3dbe6b215 Fix removed block zero size log reporting 2013-08-30 09:39:01 +08:00
Patrick Wendell abdbacf252 Merge pull request #871 from pwendell/expose-local
Expose `isLocal` in SparkContext.
2013-08-28 21:11:31 -07:00
Patrick Wendell 30d2421112 Make local variable public 2013-08-28 19:53:31 -07:00
Matei Zaharia baa84e7e4c Merge pull request #865 from tgravescs/fixtmpdir
Spark on Yarn should use yarn approved directories for spark.local.dir and tmp
2013-08-28 12:44:46 -07:00
Y.CORP.YAHOO.COM\tgraves aac1214ee4 Change Executor to only look at the env variable SPARK_YARN_MODE 2013-08-28 13:26:26 -05:00
Y.CORP.YAHOO.COM\tgraves 3f206bf0b5 Updated based on review comments. 2013-08-27 14:34:27 -05:00
Y.CORP.YAHOO.COM\tgraves cf52a3cba6 Allow for Executors to have different directories then the Spark Master for Yarn 2013-08-27 11:00:21 -05:00
Reynold Xin a77e0abb96 Added worker state to the cluster master JSON ui. 2013-08-26 11:21:03 -07:00
Reynold Xin 9db1e50344 Revert "Merge pull request #841 from rxin/json"
This reverts commit 1fb1b09928, reversing
changes made to c69c48947d.
2013-08-26 11:05:14 -07:00
Matei Zaharia 8a36fd09dd Merge pull request #854 from markhamstra/pomUpdate
Synced sbt and maven builds to use the same dependencies, etc.
2013-08-22 10:13:35 -07:00
Matei Zaharia c2d00f12e2 Merge pull request #832 from alig/coalesce
Coalesced RDD with locality
2013-08-22 10:13:03 -07:00
Mark Hamstra ff6f1b0500 Synced sbt and maven builds 2013-08-21 13:50:24 -07:00
Mark Hamstra 5eea613ec0 Removed meaningless types 2013-08-20 16:49:18 -07:00
Ali Ghodsi f20ed14e87 Merged in from upstream to use TaskLocation instead of strings 2013-08-20 16:21:43 -07:00
Ali Ghodsi 5cd21c4195 added curly braces to make the code more consistent 2013-08-20 16:16:05 -07:00
Ali Ghodsi db4bc55bef indent 2013-08-20 16:16:05 -07:00
Ali Ghodsi c0942a710f Bug in test fixed 2013-08-20 16:16:05 -07:00
Ali Ghodsi 5db41919b5 Added a test to make sure no locality preferences are ignored 2013-08-20 16:16:05 -07:00
Ali Ghodsi 7b123b3126 Simpler code 2013-08-20 16:16:05 -07:00
Ali Ghodsi 9192c358e4 simpler code 2013-08-20 16:16:05 -07:00
Ali Ghodsi a75a64eade Fixed almost all of Matei's feedback 2013-08-20 16:16:05 -07:00
Ali Ghodsi f1c853d76d fixed Matei's comments 2013-08-20 16:16:04 -07:00
Ali Ghodsi 890ea6ba79 making CoalescedRDDPartition public 2013-08-20 16:16:04 -07:00
Ali Ghodsi d6b6c680be comment in the test to make it more understandable 2013-08-20 16:16:04 -07:00
Ali Ghodsi b69e7166ba Coalescer now uses current preferred locations for derived RDDs. Made run() in DAGScheduler thread safe and added a method to be able to ask it for preferred locations. Added a similar method that wraps the former inside SparkContext. 2013-08-20 16:16:04 -07:00