Patrick Wendell
6079721fa1
Update build version in master
2013-09-24 11:41:51 -07:00
shane-huang
dfbdc9ddb7
added spark-class and spark-executor to sbin
...
Signed-off-by: shane-huang <shengsheng.huang@intel.com>
2013-09-23 11:28:58 +08:00
Prashant Sharma
383e151fd7
Merge branch 'master' of git://github.com/mesos/spark into scala-2.10
...
Conflicts:
core/src/main/scala/org/apache/spark/SparkContext.scala
project/SparkBuild.scala
2013-09-15 10:55:12 +05:30
Aaron Davidson
a3868544be
Whoopsy daisy
2013-09-08 00:30:47 -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
Aaron Davidson
b8a0b6ea5e
Memoize StorageLevels read from JVM
2013-09-06 15:36:04 -07:00
Prashant Sharma
4106ae9fbf
Merged with master
2013-09-06 17:53:01 +05:30
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
Matei Zaharia
12b2f1f9c9
Add missing license headers found with RAT
2013-09-02 12:23:03 -07:00
Matei Zaharia
141f54279e
Further fixes to get PySpark to work on Windows
2013-09-02 01:19:29 +00:00
Matei Zaharia
6550e5e60c
Allow PySpark to launch worker.py directly on Windows
2013-09-01 18:06:15 -07: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
bbaa9d7d6e
Add banner to PySpark and make wordcount output nicer
2013-09-01 14:13:16 -07:00
Matei Zaharia
46eecd110a
Initial work to rename package to org.apache.spark
2013-09-01 14:13:13 -07:00
Matei Zaharia
6edef9c833
Merge pull request #861 from AndreSchumacher/pyspark_sampling_function
...
Pyspark sampling function
2013-08-31 13:39:24 -07:00
Matei Zaharia
fd89835965
Merge pull request #870 from JoshRosen/spark-885
...
Don't send SIGINT / ctrl-c to Py4J gateway subprocess
2013-08-31 13:18:12 -07:00
Matei Zaharia
618f0ecb43
Merge pull request #869 from AndreSchumacher/subtract
...
PySpark: implementing subtractByKey(), subtract() and keyBy()
2013-08-30 18:17:13 -07:00
Andre Schumacher
96571c2524
PySpark: replacing class manifest by class tag for Scala 2.10.2 inside rdd.py
2013-08-30 15:00:42 -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
Andre Schumacher
a511c5379e
RDD sample() and takeSample() prototypes for PySpark
2013-08-28 16:46:13 -07:00
Josh Rosen
742c44eae6
Don't send SIGINT to Py4J gateway subprocess.
...
This addresses SPARK-885, a usability issue where PySpark's
Java gateway process would be killed if the user hit ctrl-c.
Note that SIGINT still won't cancel the running s
This fix is based on http://stackoverflow.com/questions/5045771
2013-08-28 16:39:44 -07:00
Andre Schumacher
457bcd3343
PySpark: implementing subtractByKey(), subtract() and keyBy()
2013-08-28 16:14:22 -07:00
Andre Schumacher
76077bf9f4
Implementing SPARK-838: Add DoubleRDDFunctions methods to PySpark
2013-08-21 17:05:58 -07:00
Andre Schumacher
c7e348faec
Implementing SPARK-878 for PySpark: adding zip and egg files to context and passing it down to workers which add these to their sys.path
2013-08-16 11:58:20 -07:00
Josh Rosen
7a9abb9ddc
Fix PySpark unit tests on Python 2.6.
2013-08-14 15:12:12 -07:00
Matei Zaharia
d3525babee
Merge pull request #813 from AndreSchumacher/add_files_pyspark
...
Implementing SPARK-865: Add the equivalent of ADD_JARS to PySpark
2013-08-12 21:02:39 -07:00
Andre Schumacher
8fd5c7bc00
Implementing SPARK-865: Add the equivalent of ADD_JARS to PySpark
...
Now ADD_FILES uses a comma as file name separator.
2013-08-12 20:22:52 -07:00
Josh Rosen
b95732632b
Do not inherit master's PYTHONPATH on workers.
...
This fixes SPARK-832, an issue where PySpark
would not work when the master and workers used
different SPARK_HOME paths.
This change may potentially break code that relied
on the master's PYTHONPATH being used on workers.
To have custom PYTHONPATH additions used on the
workers, users should set a custom PYTHONPATH in
spark-env.sh rather than setting it in the shell.
2013-07-29 22:08:57 -07:00
Matei Zaharia
feba7ee540
SPARK-815. Python parallelize() should split lists before batching
...
One unfortunate consequence of this fix is that we materialize any
collections that are given to us as generators, but this seems necessary
to get reasonable behavior on small collections. We could add a
batchSize parameter later to bypass auto-computation of batch size if
this becomes a problem (e.g. if users really want to parallelize big
generators nicely)
2013-07-29 02:51:43 -04:00
Matei Zaharia
d75c308695
Use None instead of empty string as it's slightly smaller/faster
2013-07-29 02:51:43 -04:00
Matei Zaharia
b5ec355622
Optimize Python foreach() to not return as many objects
2013-07-29 02:51:43 -04:00
Matei Zaharia
b9d6783f36
Optimize Python take() to not compute entire first partition
2013-07-29 02:51:43 -04:00
Matei Zaharia
af3c9d5042
Add Apache license headers and LICENSE and NOTICE files
2013-07-16 17:21:33 -07:00
root
ec31e68d5d
Fixed PySpark perf regression by not using socket.makefile(), and improved
...
debuggability by letting "print" statements show up in the executor's stderr
Conflicts:
core/src/main/scala/spark/api/python/PythonRDD.scala
2013-07-01 06:26:31 +00:00
Jey Kottalam
c75bed0eeb
Fix reporting of PySpark exceptions
2013-06-21 12:14:16 -04:00
Jey Kottalam
7c5ff733ee
PySpark daemon: fix deadlock, improve error handling
2013-06-21 12:14:16 -04:00
Jey Kottalam
62c4781400
Add tests and fixes for Python daemon shutdown
2013-06-21 12:14:16 -04:00
Jey Kottalam
c79a6078c3
Prefork Python worker processes
2013-06-21 12:14:16 -04:00
Jey Kottalam
40afe0d2a5
Add Python timing instrumentation
2013-06-21 12:14:16 -04:00
Jey Kottalam
9a731f5a6d
Fix Python saveAsTextFile doctest to not expect order to be preserved
2013-04-02 11:59:20 -07:00
Josh Rosen
2c966c98fb
Change numSplits to numPartitions in PySpark.
2013-02-24 13:25:09 -08:00
Mark Hamstra
b7a1fb5c5d
Add commutative requirement for 'reduce' to Python docstring.
2013-02-09 12:14:11 -08:00
Josh Rosen
e61729113d
Remove unnecessary doctest __main__ methods.
2013-02-03 21:29:40 -08:00
Josh Rosen
8fbd5380b7
Fetch fewer objects in PySpark's take() method.
2013-02-03 06:44:49 +00:00
Josh Rosen
2415c18f48
Fix reporting of PySpark doctest failures.
2013-02-03 06:44:11 +00:00
Josh Rosen
e211f405bc
Use spark.local.dir for PySpark temp files (SPARK-580).
2013-02-01 11:50:27 -08:00
Josh Rosen
9cc6ff9c4e
Do not launch JavaGateways on workers (SPARK-674).
...
The problem was that the gateway was being initialized whenever the
pyspark.context module was loaded. The fix uses lazy initialization
that occurs only when SparkContext instances are actually constructed.
I also made the gateway and jvm variables private.
This change results in ~3-4x performance improvement when running the
PySpark unit tests.
2013-02-01 11:13:10 -08:00
Josh Rosen
57b64d0d19
Fix stdout redirection in PySpark.
2013-02-01 00:25:19 -08:00
Patrick Wendell
3446d5c8d6
SPARK-673: Capture and re-throw Python exceptions
...
This patch alters the Python <-> executor protocol to pass on
exception data when they occur in user Python code.
2013-01-31 18:06:11 -08:00
Matei Zaharia
55327a283e
Merge pull request #430 from pwendell/pyspark-guide
...
Minor improvements to PySpark docs
2013-01-30 15:35:29 -08:00
Patrick Wendell
3f945e3b83
Make module help available in python shell.
...
Also, adds a line in doc explaining how to use.
2013-01-30 15:04:06 -08:00
Stephen Haberman
7dfb82a992
Replace old 'master' term with 'driver'.
2013-01-25 11:03:00 -06:00
Matei Zaharia
a2f4891d1d
Merge pull request #396 from JoshRosen/spark-653
...
Make PySpark AccumulatorParam an abstract base class
2013-01-24 13:05:03 -08:00
Josh Rosen
b47d054cfc
Remove use of abc.ABCMeta due to cloudpickle issue.
...
cloudpickle runs into issues while pickling subclasses of AccumulatorParam,
which may be related to this Python issue:
http://bugs.python.org/issue7689
This seems hard to fix and the ABCMeta wasn't necessary, so I removed it.
2013-01-23 11:47:27 -08:00
Josh Rosen
ae2ed2947d
Allow PySpark's SparkFiles to be used from driver
...
Fix minor documentation formatting issues.
2013-01-23 10:58:50 -08:00
Josh Rosen
35168d9c89
Fix sys.path bug in PySpark SparkContext.addPyFile
2013-01-22 17:54:11 -08:00
Josh Rosen
c75ae3622e
Make AccumulatorParam an abstract base class.
2013-01-21 22:32:57 -08:00
Josh Rosen
ef711902c1
Don't download files to master's working directory.
...
This should avoid exceptions caused by existing
files with different contents.
I also removed some unused code.
2013-01-21 17:34:17 -08:00
Matei Zaharia
c7b5e5f1ec
Merge pull request #389 from JoshRosen/python_rdd_checkpointing
...
Add checkpointing to the Python API
2013-01-20 17:10:44 -08:00
Josh Rosen
9f211dd3f0
Fix PythonPartitioner equality; see SPARK-654.
...
PythonPartitioner did not take the Python-side partitioning function
into account when checking for equality, which might cause problems
in the future.
2013-01-20 15:41:42 -08:00
Josh Rosen
00d70cd660
Clean up setup code in PySpark checkpointing tests
2013-01-20 15:38:11 -08:00
Josh Rosen
5b6ea9e9a0
Update checkpointing API docs in Python/Java.
2013-01-20 15:31:41 -08:00
Josh Rosen
d0ba80dc72
Add checkpointFile() and more tests to PySpark.
2013-01-20 13:59:45 -08:00
Josh Rosen
7ed1bf4b48
Add RDD checkpointing to Python API.
2013-01-20 13:19:19 -08:00
Josh Rosen
17035db159
Add __repr__ to Accumulator; fix bug in sc.accumulator
2013-01-20 11:58:57 -08:00
Matei Zaharia
a23ed25f3c
Add a class comment to Accumulator
2013-01-20 02:10:25 -08:00
Matei Zaharia
8e7f098a2c
Added accumulators to PySpark
2013-01-20 01:57:44 -08:00
Josh Rosen
49c74ba2af
Change PYSPARK_PYTHON_EXEC to PYSPARK_PYTHON.
2013-01-10 08:10:59 -08:00
Josh Rosen
b57dd0f160
Add mapPartitionsWithSplit() to PySpark.
2013-01-08 16:05:02 -08:00
Josh Rosen
33beba3965
Change PySpark RDD.take() to not call iterator().
2013-01-03 14:52:21 -08:00
Josh Rosen
ce9f1bbe20
Add pyspark
script to replace the other scripts.
...
Expand the PySpark programming guide.
2013-01-01 21:25:49 -08:00
Josh Rosen
b58340dbd9
Rename top-level 'pyspark' directory to 'python'
2013-01-01 15:05:00 -08:00