Commit graph

4825 commits

Author SHA1 Message Date
Harvey Feng 993e293d6e Merge pull request #1 from colorant/yarn-client-2.2
Port yarn-client mode for new-yarn
2013-11-27 00:57:54 -08:00
Prashant Sharma 54862af5ee Improvements from the review comments and followed Boy Scout Rule. 2013-11-27 14:26:28 +05:30
Raymond Liu 403cac9be3 Port yarn-client mode for new-yarn 2013-11-27 16:10:42 +08:00
Matei Zaharia fb6875dd5c Merge pull request #146 from JoshRosen/pyspark-custom-serializers
Custom Serializers for PySpark

This pull request adds support for custom serializers to PySpark.  For now, all Python-transformed (or parallelize()d RDDs) are serialized with the same serializer that's specified when creating SparkContext.

For now, PySpark includes `PickleSerDe` and `MarshalSerDe` classes for using Python's `pickle` and `marshal` serializers.  It's pretty easy to add support for other serializers, although I still need to add instructions on this.

A few notable changes:

- The Scala `PythonRDD` class no longer manipulates Pickled objects; data from `textFile` is written to Python as MUTF-8 strings.  The Python code performs the appropriate bookkeeping to track which deserializer should be used when reading an underlying JavaRDD.  This mechanism could also be used to support other data exchange formats, such as MsgPack.
- Several magic numbers were refactored into constants.
- Batching is implemented by wrapping / decorating an unbatched SerDe.
2013-11-26 20:55:40 -08:00
Matei Zaharia 330ada1766 Merge pull request #207 from henrydavidge/master
Log a warning if a task's serialized size is very big

As per Reynold's instructions, we now create a warning level log entry if a task's serialized size is too big. "Too big" is currently defined as 100kb. This warning message is generated at most once for each stage.
2013-11-26 19:08:33 -08:00
Matei Zaharia 615213fb82 Merge pull request #212 from markhamstra/SPARK-963
[SPARK-963] Fixed races in JobLoggerSuite
2013-11-26 19:07:20 -08:00
Harvey Feng afe4fe7f5e Merge remote-tracking branch 'origin/master' into yarn-2.2
Conflicts:
	yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
2013-11-26 15:03:03 -08:00
Harvey Feng a1a1c62a3e Add optional Hadoop 2.2 settings in sbt build.
If the Hadoop used is version 2.2 or derived from it, then Spark
will be compiled against protobuf-2.5 and a protobuf-2.5 version of
Akka 2.0.5.
2013-11-26 14:58:41 -08:00
Josh Rosen 1b74a27da0 Removed unused basestring case from dump_stream. 2013-11-26 14:35:12 -08:00
hhd 57579934f0 Emit warning when task size > 100KB 2013-11-26 16:58:39 -05:00
Mark Hamstra ed7ecb93ce [SPARK-963] Wait for SparkListenerBus eventQueue to be empty before checking jobLogger state 2013-11-26 13:30:17 -08:00
Reynold Xin cb976dfb50 Merge pull request #209 from pwendell/better-docs
Improve docs for shuffle instrumentation
2013-11-26 10:23:19 -08:00
Prashant Sharma dca946ff67 Documenting the newly added spark properties. 2013-11-26 20:47:38 +05:30
Prashant Sharma 560e44a8e1 Restored master address for client. 2013-11-26 18:18:05 +05:30
haitao.yao db998a6e14 add http timeout for httpbroadcast 2013-11-26 18:23:48 +08:00
Prashant Sharma d092a8cc6a Fixed compile time warnings and formatting post merge. 2013-11-26 15:21:50 +05:30
Matei Zaharia 18d6df0e17 Merge pull request #86 from holdenk/master
Add histogram functionality to DoubleRDDFunctions

This pull request add histogram functionality to the DoubleRDDFunctions.
2013-11-26 00:00:07 -08:00
Patrick Wendell 297c09d4bb Improve docs for shuffle instrumentation 2013-11-25 22:53:28 -08:00
Holden Karau 7222ee2977 Fix the test 2013-11-25 21:06:42 -08:00
Matei Zaharia 0e2109ddb2 Merge pull request #204 from rxin/hash
OpenHashSet fixes

Incorporated ideas from pull request #200.
- Use Murmur Hash 3 finalization step to scramble the bits of HashCode
  instead of the simpler version in java.util.HashMap; the latter one
  had trouble with ranges of consecutive integers. Murmur Hash 3 is used
  by fastutil.
- Don't check keys for equality when re-inserting due to growing the
  table; the keys will already be unique.
- Remember the grow threshold instead of recomputing it on each insert

Also added unit tests for size estimation for specialized hash sets and maps.
2013-11-25 20:48:37 -08:00
Matei Zaharia c46067f096 Merge pull request #206 from ash211/patch-2
Update tuning.md

Clarify when serializer is used based on recent user@ mailing list discussion.
2013-11-25 19:09:31 -08:00
Matei Zaharia 14bb465bb3 Merge pull request #201 from rxin/mappartitions
Use the proper partition index in mapPartitionsWIthIndex

mapPartitionsWithIndex uses TaskContext.partitionId as the partition index. TaskContext.partitionId used to be identical to the partition index in a RDD. However, pull request #186 introduced a scenario (with partition pruning) that the two can be different. This pull request uses the right partition index in all mapPartitionsWithIndex related calls.

Also removed the extra MapPartitionsWIthContextRDD and put all the mapPartitions related functionality in MapPartitionsRDD.
2013-11-25 18:50:18 -08:00
Andrew Ash 08afef37a0 Update tuning.md
Clarify when serializer is used based on recent user@ mailing list discussion.
2013-11-25 17:08:52 -08:00
Matei Zaharia eb4296c8f7 Merge pull request #101 from colorant/yarn-client-scheduler
For SPARK-527, Support spark-shell when running on YARN

sync to trunk and resubmit here

In current YARN mode approaching, the application is run in the Application Master as a user program thus the whole spark context is on remote.

This approaching won't support application that involve local interaction and need to be run on where it is launched.

So In this pull request I have a YarnClientClusterScheduler and backend added.

With this scheduler, the user application is launched locally,While the executor will be launched by YARN on remote nodes with a thin AM which only launch the executor and monitor the Driver Actor status, so that when client app is done, it can finish the YARN Application as well.

This enables spark-shell to run upon YARN.

This also enable other Spark applications to have the spark context to run locally with a master-url "yarn-client". Thus e.g. SparkPi could have the result output locally on console instead of output in the log of the remote machine where AM is running on.

Docs also updated to show how to use this yarn-client mode.
2013-11-25 15:25:29 -08:00
Prashant Sharma 44fd30d3fb Merge branch 'master' into scala-2.10-wip
Conflicts:
	core/src/main/scala/org/apache/spark/rdd/RDD.scala
	project/SparkBuild.scala
2013-11-25 18:10:54 +05:30
Prashant Sharma 489862a657 Remote death watch has a funny bug.
https://gist.github.com/ScrapCodes/4805fd84906e40b7b03d
2013-11-25 18:00:02 +05:30
Reynold Xin 466fd06475 Incorporated ideas from pull request #200.
- Use Murmur Hash 3 finalization step to scramble the bits of HashCode
  instead of the simpler version in java.util.HashMap; the latter one
  had trouble with ranges of consecutive integers. Murmur Hash 3 is used
  by fastutil.

- Don't check keys for equality when re-inserting due to growing the
  table; the keys will already be unique

- Remember the grow threshold instead of recomputing it on each insert
2013-11-25 18:27:26 +08:00
Reynold Xin 95c55df1c2 Added unit tests for size estimation for specialized hash sets and maps. 2013-11-25 18:27:06 +08:00
Prashant Sharma 77929cfeed Fine tuning defaults for akka and restored tracking of dissassociated events, for they are delivered when a remote TCP socket is closed. Also made transport failure heartbeats larger interval for it is mostly not needed. As we are using remote death watch instead. 2013-11-25 14:13:21 +05:30
Reynold Xin 62889c419c Merge pull request #203 from witgo/master
Fix Maven build for metrics-graphite
2013-11-25 11:27:45 +08:00
LiGuoqiang 989203604e Fix Maven build for metrics-graphite 2013-11-25 11:23:11 +08:00
Matei Zaharia 859d62dc2a Merge pull request #151 from russellcardullo/add-graphite-sink
Add graphite sink for metrics

This adds a metrics sink for graphite.  The sink must
be configured with the host and port of a graphite node
and optionally may be configured with a prefix that will
be prepended to all metrics that are sent to graphite.
2013-11-24 16:19:51 -08:00
Matei Zaharia 65de73c7f8 Merge pull request #185 from mkolod/random-number-generator
XORShift RNG with unit tests and benchmark

This patch was introduced to address SPARK-950 - the discussion below the ticket explains not only the rationale, but also the design and testing decisions: https://spark-project.atlassian.net/browse/SPARK-950

To run unit test, start SBT console and type:
compile
test-only org.apache.spark.util.XORShiftRandomSuite
To run benchmark, type:
project core
console
Once the Scala console starts, type:
org.apache.spark.util.XORShiftRandom.benchmark(100000000)
XORShiftRandom is also an object with a main method taking the
number of iterations as an argument, so you can also run it
from the command line.
2013-11-24 15:52:33 -08:00
Reynold Xin 972171b9d9 Merge pull request #197 from aarondav/patrick-fix
Fix 'timeWriting' stat for shuffle files

Due to concurrent git branches, changes from shuffle file consolidation patch
caused the shuffle write timing patch to no longer actually measure the time,
since it requires time be measured after the stream has been closed.
2013-11-25 07:50:46 +08:00
Reynold Xin e9ff13ec72 Consolidated both mapPartitions related RDDs into a single MapPartitionsRDD.
Also changed the semantics of the index parameter in mapPartitionsWithIndex from the partition index of the output partition to the partition index in the current RDD.
2013-11-24 17:56:43 +08:00
Reynold Xin 718cc803f7 Merge pull request #200 from mateiz/hash-fix
AppendOnlyMap fixes

- Chose a more random reshuffling step for values returned by Object.hashCode to avoid some long chaining that was happening for consecutive integers (e.g. `sc.makeRDD(1 to 100000000, 100).map(t => (t, t)).reduceByKey(_ + _).count`)
- Some other small optimizations throughout (see commit comments)
2013-11-24 11:02:02 +08:00
Matei Zaharia 9837a60234 Some other optimizations to AppendOnlyMap:
- Don't check keys for equality when re-inserting due to growing the
  table; the keys will already be unique
- Remember the grow threshold instead of recomputing it on each insert
2013-11-23 17:38:29 -08:00
Matei Zaharia 7535d7fbcb Fixes to AppendOnlyMap:
- Use Murmur Hash 3 finalization step to scramble the bits of HashCode
  instead of the simpler version in java.util.HashMap; the latter one
  had trouble with ranges of consecutive integers. Murmur Hash 3 is used
  by fastutil.
- Use Object.equals() instead of Scala's == to compare keys, because the
  latter does extra casts for numeric types (see the equals method in
  https://github.com/scala/scala/blob/master/src/library/scala/runtime/BoxesRunTime.java)
2013-11-23 17:21:37 -08:00
Harvey Feng 4f1c3fa5d7 Hadoop 2.2 YARN API migration for SPARK_HOME/new-yarn 2013-11-23 17:08:30 -08:00
Harvey Feng ab8652f2d3 Add a "new-yarn" directory in SPARK_HOME, intended to contain Hadoop-2.2 API changes. 2013-11-23 17:08:30 -08:00
Harvey Feng a67ebf4377 A few more style fixes in yarn package. 2013-11-23 17:08:30 -08:00
Reynold Xin 51aa9d6e99 Merge pull request #198 from ankurdave/zipPartitions-preservesPartitioning
Support preservesPartitioning in RDD.zipPartitions

In `RDD.zipPartitions`, add support for a `preservesPartitioning` option (similar to `RDD.mapPartitions`) that reuses the first RDD's partitioner.
2013-11-23 19:46:46 +08:00
Ankur Dave c1507afc6c Support preservesPartitioning in RDD.zipPartitions 2013-11-23 03:03:31 -08:00
Aaron Davidson ccea38b759 Fix 'timeWriting' stat for shuffle files
Due to concurrent git branches, changes from shuffle file consolidation patch
caused the shuffle write timing patch to no longer actually measure the time,
since it requires time be measured after the stream has been closed.
2013-11-21 21:36:08 -08:00
Reynold Xin 086b097e33 Merge pull request #193 from aoiwelle/patch-1
Fix Kryo Serializer buffer documentation inconsistency

The documentation here is inconsistent with the coded default and other documentation.
2013-11-22 10:26:39 +08:00
Reynold Xin f20093c3af Merge pull request #196 from pwendell/master
TimeTrackingOutputStream should pass on calls to close() and flush().

Without this fix you get a huge number of open files when running shuffles.
2013-11-22 10:12:13 +08:00
Raymond Liu ab3cefde53 Add YarnClientClusterScheduler and Backend.
With this scheduler, the user application is launched locally,
While the executor will be launched by YARN on remote nodes.

This enables spark-shell to run upon YARN.
2013-11-22 09:23:27 +08:00
Patrick Wendell 53b94ef2f5 TimeTrackingOutputStream should pass on calls to close() and flush().
Without this fix you get a huge number of open shuffles after running
shuffles.
2013-11-21 17:20:15 -08:00
Kay Ousterhout fc78f67da2 Added logging of scheduler delays to UI 2013-11-21 16:54:23 -08:00
dhardy92 92c7cc00fb [Deb] fix package of Spark classes adding org.apache prefix in scripts embeded in .deb 2013-11-21 18:51:36 +01:00