Commit graph

211 commits

Author SHA1 Message Date
Ai He ecd3aacf28 [SPARK-7810] [PYSPARK] solve python rdd socket connection problem
Method "_load_from_socket" in rdd.py cannot load data from jvm socket when ipv6 is used. The current method only works well with ipv4. New modification should work around both two protocols.

Author: Ai He <ai.he@ussuning.com>
Author: AiHe <ai.he@ussuning.com>

Closes #6338 from AiHe/pyspark-networking-issue and squashes the following commits:

d4fc9c4 [Ai He] handle code review 2
e75c5c8 [Ai He] handle code review
5644953 [AiHe] solve python rdd socket connection problem to jvm
2015-06-29 14:36:26 -07:00
Scott Taylor f0dcbe8a7c [SPARK-8541] [PYSPARK] test the absolute error in approx doctests
A minor change but one which is (presumably) visible on the public api docs webpage.

Author: Scott Taylor <github@megatron.me.uk>

Closes #6942 from megatron-me-uk/patch-3 and squashes the following commits:

fbed000 [Scott Taylor] test the absolute error in approx doctests
2015-06-22 23:37:56 -07:00
zsxwing 0fc4b96f3e [SPARK-8373] [PYSPARK] Add emptyRDD to pyspark and fix the issue when calling sum on an empty RDD
This PR fixes the sum issue and also adds `emptyRDD` so that it's easy to create a test case.

Author: zsxwing <zsxwing@gmail.com>

Closes #6826 from zsxwing/python-emptyRDD and squashes the following commits:

b36993f [zsxwing] Update the return type to JavaRDD[T]
71df047 [zsxwing] Add emptyRDD to pyspark and fix the issue when calling sum on an empty RDD
2015-06-17 13:59:39 -07:00
Sean Owen 6e53402696 [SPARK-6416] [DOCS] RDD.fold() requires the operator to be commutative
Document current limitation of rdd.fold.

This does not resolve SPARK-6416 but just documents the issue.
CC JoshRosen

Author: Sean Owen <sowen@cloudera.com>

Closes #6231 from srowen/SPARK-6416 and squashes the following commits:

9fef39f [Sean Owen] Add comment to other languages; reword to highlight the difference from non-distributed collections and to not suggest it is a bug that is to be fixed
da40d84 [Sean Owen] Document current limitation of rdd.fold.
2015-05-21 19:42:51 +01:00
Davies Liu 32fbd297dd [SPARK-6216] [PYSPARK] check python version of worker with driver
This PR revert #5404, change to pass the version of python in driver into JVM, check it in worker before deserializing closure, then it can works with different major version of Python.

Author: Davies Liu <davies@databricks.com>

Closes #6203 from davies/py_version and squashes the following commits:

b8fb76e [Davies Liu] fix test
6ce5096 [Davies Liu] use string for version
47c6278 [Davies Liu] check python version of worker with driver
2015-05-18 12:55:13 -07:00
Vinod K C dda6d9f404 [SPARK-7438] [SPARK CORE] Fixed validation of relativeSD in countApproxDistinct
Author: Vinod K C <vinod.kc@huawei.com>

Closes #5974 from vinodkc/fix_countApproxDistinct_Validation and squashes the following commits:

3a3d59c [Vinod K C] Reverted removal of validation relativeSD<0.000017
799976e [Vinod K C] Removed testcase to assert IAE when relativeSD>3.7
8ddbfae [Vinod K C] Remove blank line
b1b00a3 [Vinod K C] Removed relativeSD validation from python API,RDD.scala will do validation
122d378 [Vinod K C] Fixed validation of relativeSD in  countApproxDistinct
2015-05-09 10:03:15 +01:00
Davies Liu ab9128fb7e [SPARK-6949] [SQL] [PySpark] Support Date/Timestamp in Column expression
This PR enable auto_convert in JavaGateway, then we could register a converter for a given types, for example, date and datetime.

There are two bugs related to auto_convert, see [1] and [2], we workaround it in this PR.

[1]  https://github.com/bartdag/py4j/issues/160
[2] https://github.com/bartdag/py4j/issues/161

cc rxin JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #5570 from davies/py4j_date and squashes the following commits:

eb4fa53 [Davies Liu] fix tests in python 3
d17d634 [Davies Liu] rollback changes in mllib
2e7566d [Davies Liu] convert tuple into ArrayList
ceb3779 [Davies Liu] Update rdd.py
3c373f3 [Davies Liu] support date and datetime by auto_convert
cb094ff [Davies Liu] enable auto convert
2015-04-21 00:08:18 -07:00
Davies Liu 04e44b37cc [SPARK-4897] [PySpark] Python 3 support
This PR update PySpark to support Python 3 (tested with 3.4).

Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped.

TODO: ec2/spark-ec2.py is not fully tested with python3.

Author: Davies Liu <davies@databricks.com>
Author: twneale <twneale@gmail.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #5173 from davies/python3 and squashes the following commits:

d7d6323 [Davies Liu] fix tests
6c52a98 [Davies Liu] fix mllib test
99e334f [Davies Liu] update timeout
b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
cafd5ec [Davies Liu] adddress comments from @mengxr
bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
179fc8d [Davies Liu] tuning flaky tests
8c8b957 [Davies Liu] fix ResourceWarning in Python 3
5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
4006829 [Davies Liu] fix test
2fc0066 [Davies Liu] add python3 path
71535e9 [Davies Liu] fix xrange and divide
5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
ed498c8 [Davies Liu] fix compatibility with python 3
820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
ad7c374 [Davies Liu] fix mllib test and warning
ef1fc2f [Davies Liu] fix tests
4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
59bb492 [Davies Liu] fix tests
1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
ca0fdd3 [Davies Liu] fix code style
9563a15 [Davies Liu] add imap back for python 2
0b1ec04 [Davies Liu] make python examples work with Python 3
d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
a716d34 [Davies Liu] test with python 3.4
f1700e8 [Davies Liu] fix test in python3
671b1db [Davies Liu] fix test in python3
692ff47 [Davies Liu] fix flaky test
7b9699f [Davies Liu] invalidate import cache for Python 3.3+
9c58497 [Davies Liu] fix kill worker
309bfbf [Davies Liu] keep compatibility
5707476 [Davies Liu] cleanup, fix hash of string in 3.3+
8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3
f53e1f0 [Davies Liu] fix tests
70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3
a39167e [Davies Liu] support customize class in __main__
814c77b [Davies Liu] run unittests with python 3
7f4476e [Davies Liu] mllib tests passed
d737924 [Davies Liu] pass ml tests
375ea17 [Davies Liu] SQL tests pass
6cc42a9 [Davies Liu] rename
431a8de [Davies Liu] streaming tests pass
78901a7 [Davies Liu] fix hash of serializer in Python 3
24b2f2e [Davies Liu] pass all RDD tests
35f48fe [Davies Liu] run future again
1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py
6e3c21d [Davies Liu] make cloudpickle work with Python3
2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run
1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out
7354371 [twneale] buffer --> memoryview  I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work.
b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?).
f40d925 [twneale] xrange --> range
e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206
79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper
2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3
854be27 [Josh Rosen] Run `futurize` on Python code:
7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py.
2015-04-16 16:20:57 -07:00
Davies Liu f11288d527 [SPARK-6886] [PySpark] fix big closure with shuffle
Currently, the created broadcast object will have same life cycle as RDD in Python. For multistage jobs, an PythonRDD will be created in JVM and the RDD in Python may be GCed, then the broadcast will be destroyed in JVM before the PythonRDD.

This PR change to use PythonRDD to track the lifecycle of the broadcast object. It also have a refactor about getNumPartitions() to avoid unnecessary creation of PythonRDD, which could be heavy.

cc JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #5496 from davies/big_closure and squashes the following commits:

9a0ea4c [Davies Liu] fix big closure with shuffle
2015-04-15 12:58:02 -07:00
Davies Liu 4740d6a158 [SPARK-6216] [PySpark] check the python version in worker
Author: Davies Liu <davies@databricks.com>

Closes #5404 from davies/check_version and squashes the following commits:

e559248 [Davies Liu] add tests
ec33b5f [Davies Liu] check the python version in worker
2015-04-10 14:04:53 -07:00
Milan Straka 0375134f42 [SPARK-5969][PySpark] Fix descending pyspark.rdd.sortByKey.
The samples should always be sorted in ascending order, because bisect.bisect_left is used on it. The reverse order of the result is already achieved in rangePartitioner by reversing the found index.

The current implementation also work, but always uses only two partitions -- the first one and the last one (because the bisect_left return returns either "beginning" or "end" for a descending sequence).

Author: Milan Straka <fox@ucw.cz>

This patch had conflicts when merged, resolved by
Committer: Josh Rosen <joshrosen@databricks.com>

Closes #4761 from foxik/fix-descending-sort and squashes the following commits:

95896b5 [Milan Straka] Add regression test for SPARK-5969.
5757490 [Milan Straka] Fix descending pyspark.rdd.sortByKey.
2015-04-10 13:50:32 -07:00
Davies Liu b5c51c8df4 [SPARK-3074] [PySpark] support groupByKey() with single huge key
This patch change groupByKey() to use external sort based approach, so it can support single huge key.

For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach).

During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy().

Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>

Closes #1977 from davies/groupby and squashes the following commits:

af3713a [Davies Liu] make sure it's iterator
67772dd [Davies Liu] fix tests
e78c15c [Davies Liu] address comments
0b0fde8 [Davies Liu] address comments
0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable
e3b8eab [Davies Liu] fix narrow dependency
2a1857a [Davies Liu] typo
d2f053b [Davies Liu] add repr for FlattedValuesSerializer
c6a2f8d [Davies Liu] address comments
9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
2b9c261 [Davies Liu] fix typo in comments
70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
ab5515b [Davies Liu] Merge branch 'master' into groupby
651f891 [Davies Liu] simplify GroupByKey
1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
1f69f93 [Davies Liu] fix tests
0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
341f1e0 [Davies Liu] add comments, refactor
47918b8 [Davies Liu] remove unused code
6540948 [Davies Liu] address comments:
17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby
4d4bc86 [Davies Liu] bugfix
8ef965e [Davies Liu] Merge branch 'master' into groupby
fbc504a [Davies Liu] Merge branch 'master' into groupby
779ed03 [Davies Liu] fix merge conflict
2c1d05b [Davies Liu] refactor, minor turning
b48cda5 [Davies Liu] Merge branch 'master' into groupby
85138e6 [Davies Liu] Merge branch 'master' into groupby
acd8e1b [Davies Liu] fix memory when groupByKey().count()
905b233 [Davies Liu] Merge branch 'sort' into groupby
1f075ed [Davies Liu] Merge branch 'master' into sort
4b07d39 [Davies Liu] compress the data while spilling
0a081c6 [Davies Liu] Merge branch 'master' into groupby
f157fe7 [Davies Liu] Merge branch 'sort' into groupby
eb53ca6 [Davies Liu] Merge branch 'master' into sort
b2dc3bf [Davies Liu] Merge branch 'sort' into groupby
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
11ba318 [Davies Liu] typo
085aef8 [Davies Liu] Merge branch 'master' into groupby
3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data
1ea0669 [Davies Liu] choose sort based groupByKey() automatically
b40bae7 [Davies Liu] bugfix
efa23df [Davies Liu] refactor, add spark.shuffle.sort=False
250be4e [Davies Liu] flatten the combined values when dumping into disks
d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting
083d842 [Davies Liu] sorted based groupByKey()
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2015-04-09 17:07:23 -07:00
Davies Liu 0cce5451ad [SPARK-6667] [PySpark] remove setReuseAddress
The reused address on server side had caused the server can not acknowledge the connected connections, remove it.

This PR will retry once after timeout, it also add a timeout at client side.

Author: Davies Liu <davies@databricks.com>

Closes #5324 from davies/collect_hang and squashes the following commits:

e5a51a2 [Davies Liu] remove setReuseAddress
7977c2f [Davies Liu] do retry on client side
b838f35 [Davies Liu] retry after timeout
2015-04-02 12:18:33 -07:00
mbonaci 28bcb9e9e8 [SPARK-6370][core] Documentation: Improve all 3 docs for RDD.sample
The docs for the `sample` method were insufficient, now less so.

Author: mbonaci <mbonaci@gmail.com>

Closes #5097 from mbonaci/master and squashes the following commits:

a6a9d97 [mbonaci] [SPARK-6370][core] Documentation: Improve all 3 docs for RDD.sample method
2015-03-20 18:33:53 +00:00
Davies Liu 8767565cef [SPARK-6194] [SPARK-677] [PySpark] fix memory leak in collect()
Because circular reference between JavaObject and JavaMember, an Java object can not be released until Python GC kick in, then it will cause memory leak in collect(), which may consume lots of memory in JVM.

This PR change the way we sending collected data back into Python from local file to socket, which could avoid any disk IO during collect, also avoid any referrers of Java object in Python.

cc JoshRosen

Author: Davies Liu <davies@databricks.com>

Closes #4923 from davies/fix_collect and squashes the following commits:

d730286 [Davies Liu] address comments
24c92a4 [Davies Liu] fix style
ba54614 [Davies Liu] use socket to transfer data from JVM
9517c8f [Davies Liu] fix memory leak in collect()
2015-03-09 16:24:06 -07:00
Davies Liu f3f4c87b3d [SPARK-5944] [PySpark] fix version in Python API docs
use RELEASE_VERSION when building the Python API docs

Author: Davies Liu <davies@databricks.com>

Closes #4731 from davies/api_version and squashes the following commits:

c9744c9 [Davies Liu] Update create-release.sh
08cbc3f [Davies Liu] fix python docs
2015-02-25 15:13:34 -08:00
Davies Liu da505e5927 [SPARK-5973] [PySpark] fix zip with two RDDs with AutoBatchedSerializer
Author: Davies Liu <davies@databricks.com>

Closes #4745 from davies/fix_zip and squashes the following commits:

2124b2c [Davies Liu] Update tests.py
b5c828f [Davies Liu] increase the number of records
c1e40fd [Davies Liu] fix zip with two RDDs with AutoBatchedSerializer
2015-02-24 14:50:00 -08:00
Davies Liu c3d2b90bde [SPARK-5785] [PySpark] narrow dependency for cogroup/join in PySpark
Currently, PySpark does not support narrow dependency during cogroup/join when the two RDDs have the partitioner, another unnecessary shuffle stage will come in.

The Python implementation of cogroup/join is different than Scala one, it depends on union() and partitionBy(). This patch will try to use PartitionerAwareUnionRDD() in union(), when all the RDDs have the same partitioner. It also fix `reservePartitioner` in all the map() or mapPartitions(), then partitionBy() can skip the unnecessary shuffle stage.

Author: Davies Liu <davies@databricks.com>

Closes #4629 from davies/narrow and squashes the following commits:

dffe34e [Davies Liu] improve test, check number of stages for join/cogroup
1ed3ba2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into narrow
4d29932 [Davies Liu] address comment
cc28d97 [Davies Liu] add unit tests
940245e [Davies Liu] address comments
ff5a0a6 [Davies Liu] skip the partitionBy() on Python side
eb26c62 [Davies Liu] narrow dependency in PySpark
2015-02-17 16:54:57 -08:00
Vladimir Vladimirov b3872e00d1 SPARK-5633 pyspark saveAsTextFile support for compression codec
See https://issues.apache.org/jira/browse/SPARK-5633 for details

Author: Vladimir Vladimirov <vladimir.vladimirov@magnetic.com>

Closes #4403 from smartkiwi/master and squashes the following commits:

94c014e [Vladimir Vladimirov] SPARK-5633 pyspark saveAsTextFile support for compression codec
2015-02-06 13:55:02 -08:00
Davies Liu dc101b0e4e [SPARK-5577] Python udf for DataFrame
Author: Davies Liu <davies@databricks.com>

Closes #4351 from davies/python_udf and squashes the following commits:

d250692 [Davies Liu] fix conflict
34234d4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python_udf
440f769 [Davies Liu] address comments
f0a3121 [Davies Liu] track life cycle of broadcast
f99b2e1 [Davies Liu] address comments
462b334 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python_udf
7bccc3b [Davies Liu] python udf
58dee20 [Davies Liu] clean up
2015-02-04 15:55:09 -08:00
Xiangrui Meng 4ee79c71af [SPARK-5430] move treeReduce and treeAggregate from mllib to core
We have seen many use cases of `treeAggregate`/`treeReduce` outside the ML domain. Maybe it is time to move them to Core. pwendell

Author: Xiangrui Meng <meng@databricks.com>

Closes #4228 from mengxr/SPARK-5430 and squashes the following commits:

20ad40d [Xiangrui Meng] exclude tree* from mima
e89a43e [Xiangrui Meng] fix compile and update java doc
3ae1a4b [Xiangrui Meng] add treeReduce/treeAggregate to Python
6f948c5 [Xiangrui Meng] add treeReduce/treeAggregate to JavaRDDLike
d600b6c [Xiangrui Meng] move treeReduce and treeAggregate to core
2015-01-28 17:26:03 -08:00
Yandu Oppacher 3bead67d59 [SPARK-4387][PySpark] Refactoring python profiling code to make it extensible
This PR is based on #3255 , fix conflicts and code style.

Closes #3255.

Author: Yandu Oppacher <yandu.oppacher@jadedpixel.com>
Author: Davies Liu <davies@databricks.com>

Closes #3901 from davies/refactor-python-profile-code and squashes the following commits:

b4a9306 [Davies Liu] fix tests
4b79ce8 [Davies Liu] add docstring for profiler_cls
2700e47 [Davies Liu] use BasicProfiler as default
349e341 [Davies Liu] more refactor
6a5d4df [Davies Liu] refactor and fix tests
31bf6b6 [Davies Liu] fix code style
0864b5d [Yandu Oppacher] Remove unused method
76a6c37 [Yandu Oppacher] Added a profile collector to accumulate the profilers per stage
9eefc36 [Yandu Oppacher] Fix doc
9ace076 [Yandu Oppacher] Refactor of profiler, and moved tests around
8739aff [Yandu Oppacher] Code review fixes
9bda3ec [Yandu Oppacher] Refactor profiler code
2015-01-28 13:48:06 -08:00
Michael Nazario 456c11f15a [SPARK-5440][pyspark] Add toLocalIterator to pyspark rdd
Since Java and Scala both have access to iterate over partitions via the "toLocalIterator" function, python should also have that same ability.

Author: Michael Nazario <mnazario@palantir.com>

Closes #4237 from mnazario/feature/toLocalIterator and squashes the following commits:

1c58526 [Michael Nazario] Fix documentation off by one error
0cdc8f8 [Michael Nazario] Add toLocalIterator to PySpark
2015-01-28 12:47:12 -08:00
Sandy Ryza 406f6d3070 SPARK-5458. Refer to aggregateByKey instead of combineByKey in docs
Author: Sandy Ryza <sandy@cloudera.com>

Closes #4251 from sryza/sandy-spark-5458 and squashes the following commits:

460827a [Sandy Ryza] Python too
d2dc160 [Sandy Ryza] SPARK-5458. Refer to aggregateByKey instead of combineByKey in docs
2015-01-28 12:41:23 -08:00
Josh Rosen cef1f092a6 [SPARK-5063] More helpful error messages for several invalid operations
This patch adds more helpful error messages for invalid programs that define nested RDDs, broadcast RDDs, perform actions inside of transformations (e.g. calling `count()` from inside of `map()`), and call certain methods on stopped SparkContexts.  Currently, these invalid programs lead to confusing NullPointerExceptions at runtime and have been a major source of questions on the mailing list and StackOverflow.

In a few cases, I chose to log warnings instead of throwing exceptions in order to avoid any chance that this patch breaks programs that worked "by accident" in earlier Spark releases (e.g. programs that define nested RDDs but never run any jobs with them).

In SparkContext, the new `assertNotStopped()` method is used to check whether methods are being invoked on a stopped SparkContext.  In some cases, user programs will not crash in spite of calling methods on stopped SparkContexts, so I've only added `assertNotStopped()` calls to methods that always throw exceptions when called on stopped contexts (e.g. by dereferencing a null `dagScheduler` pointer).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3884 from JoshRosen/SPARK-5063 and squashes the following commits:

a38774b [Josh Rosen] Fix spelling typo
a943e00 [Josh Rosen] Convert two exceptions into warnings in order to avoid breaking user programs in some edge-cases.
2d0d7f7 [Josh Rosen] Fix test to reflect 1.2.1 compatibility
3f0ea0c [Josh Rosen] Revert two unintentional formatting changes
8e5da69 [Josh Rosen] Remove assertNotStopped() calls for methods that were sometimes safe to call on stopped SC's in Spark 1.2
8cff41a [Josh Rosen] IllegalStateException fix
6ef68d0 [Josh Rosen] Fix Python line length issues.
9f6a0b8 [Josh Rosen] Add improved error messages to PySpark.
13afd0f [Josh Rosen] SparkException -> IllegalStateException
8d404f3 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5063
b39e041 [Josh Rosen] Fix BroadcastSuite test which broadcasted an RDD
99cc09f [Josh Rosen] Guard against calling methods on stopped SparkContexts.
34833e8 [Josh Rosen] Add more descriptive error message.
57cc8a1 [Josh Rosen] Add error message when directly broadcasting RDD.
15b2e6b [Josh Rosen] [SPARK-5063] Useful error messages for nested RDDs and actions inside of transformations
2015-01-23 17:53:15 -08:00
Sean Owen 306ff187af SPARK-5270 [CORE] Provide isEmpty() function in RDD API
Pretty minor, but submitted for consideration -- this would at least help people make this check in the most efficient way I know.

Author: Sean Owen <sowen@cloudera.com>

Closes #4074 from srowen/SPARK-5270 and squashes the following commits:

66885b8 [Sean Owen] Add note that JavaRDDLike should not be implemented by user code
2e9b490 [Sean Owen] More tests, and Mima-exclude the new isEmpty method in JavaRDDLike
28395ff [Sean Owen] Add isEmpty to Java, Python
7dd04b7 [Sean Owen] Add efficient RDD.isEmpty()
2015-01-19 22:50:45 -08:00
lewuathe 3cd516191b [SPARK-4822] Use sphinx tags for Python doc annotations
Modify python annotations for sphinx. There is no change to build process from.
https://github.com/apache/spark/blob/master/docs/README.md

Author: lewuathe <lewuathe@me.com>

Closes #3685 from Lewuathe/sphinx-tag-for-pydoc and squashes the following commits:

88a0fd9 [lewuathe] [SPARK-4822] Fix DevelopApi and WARN tags
3d7a398 [lewuathe] [SPARK-4822] Use sphinx tags for Python doc annotations
2014-12-17 17:31:24 -08:00
Davies Liu c246b95dd2 [SPARK-4841] fix zip with textFile()
UTF8Deserializer can not be used in BatchedSerializer, so always use PickleSerializer() when change batchSize in zip().

Also, if two RDD have the same batch size already, they did not need re-serialize any more.

Author: Davies Liu <davies@databricks.com>

Closes #3706 from davies/fix_4841 and squashes the following commits:

20ce3a3 [Davies Liu] fix bug in _reserialize()
e3ebf7c [Davies Liu] add comment
379d2c8 [Davies Liu] fix zip with textFile()
2014-12-15 22:58:26 -08:00
Davies Liu d39f2e9c68 [SPARK-4477] [PySpark] remove numpy from RDDSampler
In RDDSampler, it try use numpy to gain better performance for possion(), but the number of call of random() is only (1+faction) * N in the pure python implementation of possion(), so there is no much performance gain from numpy.

numpy is not a dependent of pyspark, so it maybe introduce some problem, such as there is no numpy installed in slaves, but only installed master, as reported in SPARK-927.

It also complicate the code a lot, so we may should remove numpy from RDDSampler.

I also did some benchmark to verify that:
```
>>> from pyspark.mllib.random import RandomRDDs
>>> rdd = RandomRDDs.uniformRDD(sc, 1 << 20, 1).cache()
>>> rdd.count()  # cache it
>>> rdd.sample(True, 0.9).count()    # measure this line
```
the results:

|withReplacement      |  random  | numpy.random |
 ------- | ------------ |  -------
|True | 1.5 s|  1.4 s|
|False|  0.6 s | 0.8 s|

closes #2313

Note: this patch including some commits that not mirrored to github, it will be OK after it catches up.

Author: Davies Liu <davies@databricks.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #3351 from davies/numpy and squashes the following commits:

5c438d7 [Davies Liu] fix comment
c5b9252 [Davies Liu] Merge pull request #1 from mengxr/SPARK-4477
98eb31b [Xiangrui Meng] make poisson sampling slightly faster
ee17d78 [Davies Liu] remove = for float
13f7b05 [Davies Liu] Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/spark into numpy
f583023 [Davies Liu] fix tests
51649f5 [Davies Liu] remove numpy in RDDSampler
78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-20 16:40:25 -08:00
Davies Liu 7f22fa81eb [SPARK-4327] [PySpark] Python API for RDD.randomSplit()
```
pyspark.RDD.randomSplit(self, weights, seed=None)
    Randomly splits this RDD with the provided weights.

    :param weights: weights for splits, will be normalized if they don't sum to 1
    :param seed: random seed
    :return: split RDDs in an list

    >>> rdd = sc.parallelize(range(10), 1)
    >>> rdd1, rdd2, rdd3 = rdd.randomSplit([0.4, 0.6, 1.0], 11)
    >>> rdd1.collect()
    [3, 6]
    >>> rdd2.collect()
    [0, 5, 7]
    >>> rdd3.collect()
    [1, 2, 4, 8, 9]
```

Author: Davies Liu <davies@databricks.com>

Closes #3193 from davies/randomSplit and squashes the following commits:

78bf997 [Davies Liu] fix tests, do not use numpy in randomSplit, no performance gain
f5fdf63 [Davies Liu] fix bug with int in weights
4dfa2cd [Davies Liu] refactor
f866bcf [Davies Liu] remove unneeded change
c7a2007 [Davies Liu] switch to python implementation
95a48ac [Davies Liu] Merge branch 'master' of github.com:apache/spark into randomSplit
0d9b256 [Davies Liu] refactor
1715ee3 [Davies Liu] address comments
41fce54 [Davies Liu] randomSplit()
2014-11-18 16:37:35 -08:00
Davies Liu 7779109796 [SPARK-4304] [PySpark] Fix sort on empty RDD
This PR fix sortBy()/sortByKey() on empty RDD.

This should be back ported into 1.1/1.2

Author: Davies Liu <davies@databricks.com>

Closes #3162 from davies/fix_sort and squashes the following commits:

84f64b7 [Davies Liu] add tests
52995b5 [Davies Liu] fix sortByKey() on empty RDD
2014-11-07 20:53:03 -08:00
Davies Liu e4f42631a6 [SPARK-3886] [PySpark] simplify serializer, use AutoBatchedSerializer by default.
This PR simplify serializer, always use batched serializer (AutoBatchedSerializer as default), even batch size is 1.

Author: Davies Liu <davies@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Josh Rosen <joshrosen@databricks.com>

Closes #2920 from davies/fix_autobatch and squashes the following commits:

e544ef9 [Davies Liu] revert unrelated change
6880b14 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
1d557fc [Davies Liu] fix tests
8180907 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
76abdce [Davies Liu] clean up
53fa60b [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
d7ac751 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
2cc2497 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch
b4292ce [Davies Liu] fix bug in master
d79744c [Davies Liu] recover hive tests
be37ece [Davies Liu] refactor
eb3938d [Davies Liu] refactor serializer in scala
8d77ef2 [Davies Liu] simplify serializer, use AutoBatchedSerializer by default.
2014-11-03 23:56:14 -08:00
Xiangrui Meng 3cca196220 [SPARK-4148][PySpark] fix seed distribution and add some tests for rdd.sample
The current way of seed distribution makes the random sequences from partition i and i+1 offset by 1.

~~~
In [14]: import random

In [15]: r1 = random.Random(10)

In [16]: r1.randint(0, 1)
Out[16]: 1

In [17]: r1.random()
Out[17]: 0.4288890546751146

In [18]: r1.random()
Out[18]: 0.5780913011344704

In [19]: r2 = random.Random(10)

In [20]: r2.randint(0, 1)
Out[20]: 1

In [21]: r2.randint(0, 1)
Out[21]: 0

In [22]: r2.random()
Out[22]: 0.5780913011344704
~~~

Note: The new tests are not for this bug fix.

Author: Xiangrui Meng <meng@databricks.com>

Closes #3010 from mengxr/SPARK-4148 and squashes the following commits:

869ae4b [Xiangrui Meng] move tests tests.py
c1bacd9 [Xiangrui Meng] fix seed distribution and add some tests for rdd.sample
2014-11-03 12:24:24 -08:00
Xiangrui Meng f1e7361f66 [SPARK-4150][PySpark] return self in rdd.setName
Then we can do `rdd.setName('abc').cache().count()`.

Author: Xiangrui Meng <meng@databricks.com>

Closes #3011 from mengxr/rdd-setname and squashes the following commits:

10d0d60 [Xiangrui Meng] update test
4ac3bbd [Xiangrui Meng] return self in rdd.setName
2014-10-31 12:07:48 -07:00
yingjieMiao 49bbdcb660 [Spark] RDD take() method: overestimate too much
In the comment (Line 1083), it says: "Otherwise, interpolate the number of partitions we need to try, but overestimate it by 50%."

`(1.5 * num * partsScanned / buf.size).toInt` is the guess of "num of total partitions needed". In every iteration, we should consider the increment `(1.5 * num * partsScanned / buf.size).toInt - partsScanned`
Existing implementation 'exponentially' grows `partsScanned ` ( roughly: `x_{n+1} >= (1.5 + 1) x_n`)

This could be a performance problem. (unless this is the intended behavior)

Author: yingjieMiao <yingjie@42go.com>

Closes #2648 from yingjieMiao/rdd_take and squashes the following commits:

d758218 [yingjieMiao] scala style fix
a8e74bb [yingjieMiao] python style fix
4b6e777 [yingjieMiao] infix operator style fix
4391d3b [yingjieMiao] typo fix.
692f4e6 [yingjieMiao] cap numPartsToTry
c4483dc [yingjieMiao] style fix
1d2c410 [yingjieMiao] also change in rdd.py and AsyncRDD
d31ff7e [yingjieMiao] handle the edge case after 1 iteration
a2aa36b [yingjieMiao] RDD take method: overestimate too much
2014-10-13 13:11:55 -07:00
cocoatomo 7a3f589ef8 [SPARK-3909][PySpark][Doc] A corrupted format in Sphinx documents and building warnings
Sphinx documents contains a corrupted ReST format and have some warnings.

The purpose of this issue is same as https://issues.apache.org/jira/browse/SPARK-3773.

commit: 0e8203f4fb

output
```
$ cd ./python/docs
$ make clean html
rm -rf _build/*
sphinx-build -b html -d _build/doctrees   . _build/html
Making output directory...
Running Sphinx v1.2.3
loading pickled environment... not yet created
building [html]: targets for 4 source files that are out of date
updating environment: 4 added, 0 changed, 0 removed
reading sources... [100%] pyspark.sql
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/feature.py:docstring of pyspark.mllib.feature.Word2VecModel.findSynonyms:4: WARNING: Field list ends without a blank line; unexpected unindent.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/feature.py:docstring of pyspark.mllib.feature.Word2VecModel.transform:3: WARNING: Field list ends without a blank line; unexpected unindent.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/sql.py:docstring of pyspark.sql:4: WARNING: Bullet list ends without a blank line; unexpected unindent.
looking for now-outdated files... none found
pickling environment... done
checking consistency... done
preparing documents... done
writing output... [100%] pyspark.sql
writing additional files... (12 module code pages) _modules/index search
copying static files... WARNING: html_static_path entry u'/Users/<user>/MyRepos/Scala/spark/python/docs/_static' does not exist
done
copying extra files... done
dumping search index... done
dumping object inventory... done
build succeeded, 4 warnings.

Build finished. The HTML pages are in _build/html.
```

Author: cocoatomo <cocoatomo77@gmail.com>

Closes #2766 from cocoatomo/issues/3909-sphinx-build-warnings and squashes the following commits:

2c7faa8 [cocoatomo] [SPARK-3909][PySpark][Doc] A corrupted format in Sphinx documents and building warnings
2014-10-11 11:51:59 -07:00
Davies Liu 798ed22c28 [SPARK-3412] [PySpark] Replace Epydoc with Sphinx to generate Python API docs
Retire Epydoc, use Sphinx to generate API docs.

Refine Sphinx docs, also convert some docstrings into Sphinx style.

It looks like:
![api doc](https://cloud.githubusercontent.com/assets/40902/4538272/9e2d4f10-4dec-11e4-8d96-6e45a8fe51f9.png)

Author: Davies Liu <davies.liu@gmail.com>

Closes #2689 from davies/docs and squashes the following commits:

bf4a0a5 [Davies Liu] fix links
3fb1572 [Davies Liu] fix _static in jekyll
65a287e [Davies Liu] fix scripts and logo
8524042 [Davies Liu] Merge branch 'master' of github.com:apache/spark into docs
d5b874a [Davies Liu] Merge branch 'master' of github.com:apache/spark into docs
4bc1c3c [Davies Liu] refactor
746d0b6 [Davies Liu] @param -> :param
240b393 [Davies Liu] replace epydoc with sphinx doc
2014-10-07 18:09:27 -07:00
cocoatomo 2300eb58ae [SPARK-3773][PySpark][Doc] Sphinx build warning
When building Sphinx documents for PySpark, we have 12 warnings.
Their causes are almost docstrings in broken ReST format.

To reproduce this issue, we should run following commands on the commit: 6e27cb630d.

```bash
$ cd ./python/docs
$ make clean html
...
/Users/<user>/MyRepos/Scala/spark/python/pyspark/__init__.py:docstring of pyspark.SparkContext.sequenceFile:4: ERROR: Unexpected indentation.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/__init__.py:docstring of pyspark.RDD.saveAsSequenceFile:4: ERROR: Unexpected indentation.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.LogisticRegressionWithSGD.train:14: ERROR: Unexpected indentation.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.LogisticRegressionWithSGD.train:16: WARNING: Definition list ends without a blank line; unexpected unindent.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.LogisticRegressionWithSGD.train:17: WARNING: Block quote ends without a blank line; unexpected unindent.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.SVMWithSGD.train:14: ERROR: Unexpected indentation.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.SVMWithSGD.train:16: WARNING: Definition list ends without a blank line; unexpected unindent.
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.SVMWithSGD.train:17: WARNING: Block quote ends without a blank line; unexpected unindent.
/Users/<user>/MyRepos/Scala/spark/python/docs/pyspark.mllib.rst:50: WARNING: missing attribute mentioned in :members: or __all__: module pyspark.mllib.regression, attribute RidgeRegressionModelLinearRegressionWithSGD
/Users/<user>/MyRepos/Scala/spark/python/pyspark/mllib/tree.py:docstring of pyspark.mllib.tree.DecisionTreeModel.predict:3: ERROR: Unexpected indentation.
...
checking consistency... /Users/<user>/MyRepos/Scala/spark/python/docs/modules.rst:: WARNING: document isn't included in any toctree
...
copying static files... WARNING: html_static_path entry u'/Users/<user>/MyRepos/Scala/spark/python/docs/_static' does not exist
...
build succeeded, 12 warnings.
```

Author: cocoatomo <cocoatomo77@gmail.com>

Closes #2653 from cocoatomo/issues/3773-sphinx-build-warnings and squashes the following commits:

6f65661 [cocoatomo] [SPARK-3773][PySpark][Doc] Sphinx build warning
2014-10-06 14:08:40 -07:00
Davies Liu abf588f47a [SPARK-3749] [PySpark] fix bugs in broadcast large closure of RDD
1. broadcast is triggle unexpected
2. fd is leaked in JVM (also leak in parallelize())
3. broadcast is not unpersisted in JVM after RDD is not be used any more.

cc JoshRosen , sorry for these stupid bugs.

Author: Davies Liu <davies.liu@gmail.com>

Closes #2603 from davies/fix_broadcast and squashes the following commits:

080a743 [Davies Liu] fix bugs in broadcast large closure of RDD
2014-10-01 11:21:34 -07:00
Davies Liu c5414b6818 [SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:

```
============================================================
Profile of RDD<id=3>
============================================================
         5146507 function calls (5146487 primitive calls) in 71.094 seconds

   Ordered by: internal time, cumulative time

   ncalls  tottime  percall  cumtime  percall filename:lineno(function)
  5144576   68.331    0.000   68.331    0.000 statcounter.py:44(merge)
       20    2.735    0.137   71.071    3.554 statcounter.py:33(__init__)
       20    0.017    0.001    0.017    0.001 {cPickle.dumps}
     1024    0.003    0.000    0.003    0.000 t.py:16(<lambda>)
       20    0.001    0.000    0.001    0.000 {reduce}
       21    0.001    0.000    0.001    0.000 {cPickle.loads}
       20    0.001    0.000    0.001    0.000 copy_reg.py:95(_slotnames)
       41    0.001    0.000    0.001    0.000 serializers.py:461(read_int)
       40    0.001    0.000    0.002    0.000 serializers.py:179(_batched)
       62    0.000    0.000    0.000    0.000 {method 'read' of 'file' objects}
       20    0.000    0.000   71.072    3.554 rdd.py:863(<lambda>)
       20    0.000    0.000    0.001    0.000 serializers.py:198(load_stream)
    40/20    0.000    0.000   71.072    3.554 rdd.py:2093(pipeline_func)
       41    0.000    0.000    0.002    0.000 serializers.py:130(load_stream)
       40    0.000    0.000   71.072    1.777 rdd.py:304(func)
       20    0.000    0.000   71.094    3.555 worker.py:82(process)
```

Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as

```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
         284 function calls (276 primitive calls) in 0.001 seconds

   Ordered by: internal time, cumulative time

   ncalls  tottime  percall  cumtime  percall filename:lineno(function)
        4    0.000    0.000    0.000    0.000 serializers.py:198(load_stream)
        4    0.000    0.000    0.000    0.000 {reduce}
     12/4    0.000    0.000    0.001    0.000 rdd.py:2092(pipeline_func)
        4    0.000    0.000    0.000    0.000 {cPickle.loads}
        4    0.000    0.000    0.000    0.000 {cPickle.dumps}
      104    0.000    0.000    0.000    0.000 rdd.py:852(<genexpr>)
        8    0.000    0.000    0.000    0.000 serializers.py:461(read_int)
       12    0.000    0.000    0.000    0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".

Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"

This is bugfix of #2351 cc JoshRosen

Author: Davies Liu <davies.liu@gmail.com>

Closes #2556 from davies/profiler and squashes the following commits:

e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 18:24:57 -07:00
Josh Rosen f872e4fb80 Revert "[SPARK-3478] [PySpark] Profile the Python tasks"
This reverts commit 1aa549ba98.
2014-09-26 14:47:14 -07:00
Davies Liu 1aa549ba98 [SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:

```
============================================================
Profile of RDD<id=3>
============================================================
         5146507 function calls (5146487 primitive calls) in 71.094 seconds

   Ordered by: internal time, cumulative time

   ncalls  tottime  percall  cumtime  percall filename:lineno(function)
  5144576   68.331    0.000   68.331    0.000 statcounter.py:44(merge)
       20    2.735    0.137   71.071    3.554 statcounter.py:33(__init__)
       20    0.017    0.001    0.017    0.001 {cPickle.dumps}
     1024    0.003    0.000    0.003    0.000 t.py:16(<lambda>)
       20    0.001    0.000    0.001    0.000 {reduce}
       21    0.001    0.000    0.001    0.000 {cPickle.loads}
       20    0.001    0.000    0.001    0.000 copy_reg.py:95(_slotnames)
       41    0.001    0.000    0.001    0.000 serializers.py:461(read_int)
       40    0.001    0.000    0.002    0.000 serializers.py:179(_batched)
       62    0.000    0.000    0.000    0.000 {method 'read' of 'file' objects}
       20    0.000    0.000   71.072    3.554 rdd.py:863(<lambda>)
       20    0.000    0.000    0.001    0.000 serializers.py:198(load_stream)
    40/20    0.000    0.000   71.072    3.554 rdd.py:2093(pipeline_func)
       41    0.000    0.000    0.002    0.000 serializers.py:130(load_stream)
       40    0.000    0.000   71.072    1.777 rdd.py:304(func)
       20    0.000    0.000   71.094    3.555 worker.py:82(process)
```

Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as

```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
         284 function calls (276 primitive calls) in 0.001 seconds

   Ordered by: internal time, cumulative time

   ncalls  tottime  percall  cumtime  percall filename:lineno(function)
        4    0.000    0.000    0.000    0.000 serializers.py:198(load_stream)
        4    0.000    0.000    0.000    0.000 {reduce}
     12/4    0.000    0.000    0.001    0.000 rdd.py:2092(pipeline_func)
        4    0.000    0.000    0.000    0.000 {cPickle.loads}
        4    0.000    0.000    0.000    0.000 {cPickle.dumps}
      104    0.000    0.000    0.000    0.000 rdd.py:852(<genexpr>)
        8    0.000    0.000    0.000    0.000 serializers.py:461(read_int)
       12    0.000    0.000    0.000    0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".

Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"

Author: Davies Liu <davies.liu@gmail.com>

Closes #2351 from davies/profiler and squashes the following commits:

7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-26 09:27:42 -07:00
Aaron Staple 8ca4ecb6a5 [SPARK-546] Add full outer join to RDD and DStream.
leftOuterJoin and rightOuterJoin are already implemented.  This patch adds fullOuterJoin.

Author: Aaron Staple <aaron.staple@gmail.com>

Closes #1395 from staple/SPARK-546 and squashes the following commits:

1f5595c [Aaron Staple] Fix python style
7ac0aa9 [Aaron Staple] [SPARK-546] Add full outer join to RDD and DStream.
3b5d137 [Aaron Staple] In JavaPairDStream, make class tag specification in rightOuterJoin consistent with other functions.
31f2956 [Aaron Staple] Fix left outer join documentation comments.
2014-09-24 20:39:09 -07:00
Davies Liu fce5e251d6 [SPARK-3491] [MLlib] [PySpark] use pickle to serialize data in MLlib
Currently, we serialize the data between JVM and Python case by case manually, this cannot scale to support so many APIs in MLlib.

This patch will try to address this problem by serialize the data using pickle protocol, using Pyrolite library to serialize/deserialize in JVM. Pickle protocol can be easily extended to support customized class.

All the modules are refactored to use this protocol.

Known issues: There will be some performance regression (both CPU and memory, the serialized data increased)

Author: Davies Liu <davies.liu@gmail.com>

Closes #2378 from davies/pickle_mllib and squashes the following commits:

dffbba2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into pickle_mllib
810f97f [Davies Liu] fix equal of matrix
032cd62 [Davies Liu] add more type check and conversion for user_product
bd738ab [Davies Liu] address comments
e431377 [Davies Liu] fix cache of rdd, refactor
19d0967 [Davies Liu] refactor Picklers
2511e76 [Davies Liu] cleanup
1fccf1a [Davies Liu] address comments
a2cc855 [Davies Liu] fix tests
9ceff73 [Davies Liu] test size of serialized Rating
44e0551 [Davies Liu] fix cache
a379a81 [Davies Liu] fix pickle array in python2.7
df625c7 [Davies Liu] Merge commit '154d141' into pickle_mllib
154d141 [Davies Liu] fix autobatchedpickler
44736d7 [Davies Liu] speed up pickling array in Python 2.7
e1d1bfc [Davies Liu] refactor
708dc02 [Davies Liu] fix tests
9dcfb63 [Davies Liu] fix style
88034f0 [Davies Liu] rafactor, address comments
46a501e [Davies Liu] choose batch size automatically
df19464 [Davies Liu] memorize the module and class name during pickleing
f3506c5 [Davies Liu] Merge branch 'master' into pickle_mllib
722dd96 [Davies Liu] cleanup _common.py
0ee1525 [Davies Liu] remove outdated tests
b02e34f [Davies Liu] remove _common.py
84c721d [Davies Liu] Merge branch 'master' into pickle_mllib
4d7963e [Davies Liu] remove muanlly serialization
6d26b03 [Davies Liu] fix tests
c383544 [Davies Liu] classification
f2a0856 [Davies Liu] mllib/regression
d9f691f [Davies Liu] mllib/util
cccb8b1 [Davies Liu] mllib/tree
8fe166a [Davies Liu] Merge branch 'pickle' into pickle_mllib
aa2287e [Davies Liu] random
f1544c4 [Davies Liu] refactor clustering
52d1350 [Davies Liu] use new protocol in mllib/stat
b30ef35 [Davies Liu] use pickle to serialize data for mllib/recommendation
f44f771 [Davies Liu] enable tests about array
3908f5c [Davies Liu] Merge branch 'master' into pickle
c77c87b [Davies Liu] cleanup debugging code
60e4e2f [Davies Liu] support unpickle array.array for Python 2.6
2014-09-19 15:01:11 -07:00
Davies Liu e77fa81a61 [SPARK-3554] [PySpark] use broadcast automatically for large closure
Py4j can not handle large string efficiently, so we should use broadcast for large closure automatically. (Broadcast use local filesystem to pass through data).

Author: Davies Liu <davies.liu@gmail.com>

Closes #2417 from davies/command and squashes the following commits:

fbf4e97 [Davies Liu] bugfix
aefd508 [Davies Liu] use broadcast automatically for large closure
2014-09-18 18:11:48 -07:00
Matthew Farrellee 9d5fa763d8 [SPARK-3519] add distinct(n) to PySpark
Added missing rdd.distinct(numPartitions) and associated tests

Author: Matthew Farrellee <matt@redhat.com>

Closes #2383 from mattf/SPARK-3519 and squashes the following commits:

30b837a [Matthew Farrellee] Combine test cases to save on JVM startups
6bc4a2c [Matthew Farrellee] [SPARK-3519] add distinct(n) to SchemaRDD in PySpark
7a17f2b [Matthew Farrellee] [SPARK-3519] add distinct(n) to PySpark
2014-09-16 11:39:57 -07:00
Aaron Staple 60050f4288 [SPARK-1087] Move python traceback utilities into new traceback_utils.py file.
Also made some cosmetic cleanups.

Author: Aaron Staple <aaron.staple@gmail.com>

Closes #2385 from staple/SPARK-1087 and squashes the following commits:

7b3bb13 [Aaron Staple] Address review comments, cosmetic cleanups.
10ba6e1 [Aaron Staple] [SPARK-1087] Move python traceback utilities into new traceback_utils.py file.
2014-09-15 19:28:17 -07:00
RJ Nowling 533377621f [PySpark] Add blank line so that Python RDD.top() docstring renders correctly
Author: RJ Nowling <rnowling@gmail.com>

Closes #2370 from rnowling/python_rdd_docstrings and squashes the following commits:

5230574 [RJ Nowling] Add blank line so that Python RDD.top() docstring renders correctly
2014-09-12 09:46:21 -07:00
Sandy Ryza 16a73c2473 SPARK-2978. Transformation with MR shuffle semantics
I didn't add this to the transformations list in the docs because it's kind of obscure, but would be happy to do so if others think it would be helpful.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #2274 from sryza/sandy-spark-2978 and squashes the following commits:

4a5332a [Sandy Ryza] Fix Java test
c04b447 [Sandy Ryza] Fix Python doc and add back deleted code
433ad5b [Sandy Ryza] Add Java test
4c25a54 [Sandy Ryza] Add s at the end and a couple other fixes
9b0ba99 [Sandy Ryza] Fix compilation
36e0571 [Sandy Ryza] Fix import ordering
48c12c2 [Sandy Ryza] Add Java version and additional doc
e5381cd [Sandy Ryza] Fix python style warnings
f147634 [Sandy Ryza] SPARK-2978. Transformation with MR shuffle semantics
2014-09-08 11:20:00 -07:00
Davies Liu 110fb8b24d [SPARK-2334] fix AttributeError when call PipelineRDD.id()
The underline JavaRDD for PipelineRDD is created lazily, it's delayed until call _jrdd.

The id of JavaRDD is cached as `_id`, it saves a RPC call in py4j for later calls.

closes #1276

Author: Davies Liu <davies.liu@gmail.com>

Closes #2296 from davies/id and squashes the following commits:

e197958 [Davies Liu] fix style
9721716 [Davies Liu] fix id of PipelineRDD
2014-09-06 16:12:29 -07:00
Holden Karau da35330e83 Spark-3406 add a default storage level to python RDD persist API
Author: Holden Karau <holden@pigscanfly.ca>

Closes #2280 from holdenk/SPARK-3406-Python-RDD-persist-api-does-not-have-default-storage-level and squashes the following commits:

33eaade [Holden Karau] As Josh pointed out, sql also override persist. Make persist behave the same as in the underlying RDD as well
e658227 [Holden Karau] Fix the test I added
e95a6c5 [Holden Karau] The Python persist function did not have a default storageLevel unlike the Scala API. Noticed this issue because we got a bug report back from the book where we had documented it as if it was the same as the Scala API
2014-09-06 14:49:25 -07:00
Andrew Ash ba5bcaddec SPARK-3211 .take() is OOM-prone with empty partitions
Instead of jumping straight from 1 partition to all partitions, do exponential
growth and double the number of partitions to attempt each time instead.

Fix proposed by Paul Nepywoda

Author: Andrew Ash <andrew@andrewash.com>

Closes #2117 from ash211/SPARK-3211 and squashes the following commits:

8b2299a [Andrew Ash] Quadruple instead of double for a minor speedup
e5f7e4d [Andrew Ash] Update comment to better reflect what we're doing
09a27f7 [Andrew Ash] Update PySpark to be less OOM-prone as well
3a156b8 [Andrew Ash] SPARK-3211 .take() is OOM-prone with empty partitions
2014-09-05 18:52:05 -07:00
Davies Liu 6481d27425 [SPARK-3309] [PySpark] Put all public API in __all__
Put all public API in __all__, also put them all in pyspark.__init__.py, then we can got all the documents for public API by `pydoc pyspark`. It also can be used by other programs (such as Sphinx or Epydoc) to generate only documents for public APIs.

Author: Davies Liu <davies.liu@gmail.com>

Closes #2205 from davies/public and squashes the following commits:

c6c5567 [Davies Liu] fix message
f7b35be [Davies Liu] put SchemeRDD, Row in pyspark.sql module
7e3016a [Davies Liu] add __all__ in mllib
6281b48 [Davies Liu] fix doc for SchemaRDD
6caab21 [Davies Liu] add public interfaces into pyspark.__init__.py
2014-09-03 11:49:45 -07:00
Davies Liu e2c901b4c7 [SPARK-2871] [PySpark] add countApproxDistinct() API
RDD.countApproxDistinct(relativeSD=0.05):

        :: Experimental ::
        Return approximate number of distinct elements in the RDD.

        The algorithm used is based on streamlib's implementation of
        "HyperLogLog in Practice: Algorithmic Engineering of a State
        of The Art Cardinality Estimation Algorithm", available
        <a href="http://dx.doi.org/10.1145/2452376.2452456">here</a>.

        This support all the types of objects, which is supported by
        Pyrolite, nearly all builtin types.

        param relativeSD Relative accuracy. Smaller values create
                           counters that require more space.
                           It must be greater than 0.000017.

        >>> n = sc.parallelize(range(1000)).map(str).countApproxDistinct()
        >>> 950 < n < 1050
        True
        >>> n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct()
        >>> 18 < n < 22
        True

Author: Davies Liu <davies.liu@gmail.com>

Closes #2142 from davies/countApproxDistinct and squashes the following commits:

e20da47 [Davies Liu] remove the correction in Python
c38c4e4 [Davies Liu] fix doc tests
2ab157c [Davies Liu] fix doc tests
9d2565f [Davies Liu] add commments and link for hash collision correction
d306492 [Davies Liu] change range of hash of tuple to [0, maxint]
ded624f [Davies Liu] calculate hash in Python
4cba98f [Davies Liu] add more tests
a85a8c6 [Davies Liu] Merge branch 'master' into countApproxDistinct
e97e342 [Davies Liu] add countApproxDistinct()
2014-09-02 15:47:47 -07:00
Davies Liu 4fa2fda88f [SPARK-2871] [PySpark] add RDD.lookup(key)
RDD.lookup(key)

        Return the list of values in the RDD for key `key`. This operation
        is done efficiently if the RDD has a known partitioner by only
        searching the partition that the key maps to.

        >>> l = range(1000)
        >>> rdd = sc.parallelize(zip(l, l), 10)
        >>> rdd.lookup(42)  # slow
        [42]
        >>> sorted = rdd.sortByKey()
        >>> sorted.lookup(42)  # fast
        [42]

It also clean up the code in RDD.py, and fix several bugs (related to preservesPartitioning).

Author: Davies Liu <davies.liu@gmail.com>

Closes #2093 from davies/lookup and squashes the following commits:

1789cd4 [Davies Liu] `f` in foreach could be generator or not.
2871b80 [Davies Liu] Merge branch 'master' into lookup
c6390ea [Davies Liu] address all comments
0f1bce8 [Davies Liu] add test case for lookup()
be0e8ba [Davies Liu] fix preservesPartitioning
eb1305d [Davies Liu] add RDD.lookup(key)
2014-08-27 13:18:33 -07:00
Davies Liu f1e71d4c3b [SPARK-3073] [PySpark] use external sort in sortBy() and sortByKey()
Using external sort to support sort large datasets in reduce stage.

Author: Davies Liu <davies.liu@gmail.com>

Closes #1978 from davies/sort and squashes the following commits:

bbcd9ba [Davies Liu] check spilled bytes in tests
b125d2f [Davies Liu] add test for external sort in rdd
eae0176 [Davies Liu] choose different disks from different processes and instances
1f075ed [Davies Liu] Merge branch 'master' into sort
eb53ca6 [Davies Liu] Merge branch 'master' into sort
644abaf [Davies Liu] add license in LICENSE
19f7873 [Davies Liu] improve tests
55602ee [Davies Liu] use external sort in sortBy() and sortByKey()
2014-08-26 16:57:40 -07:00
Davies Liu 3cedc4f4d7 [SPARK-2871] [PySpark] add histgram() API
RDD.histogram(buckets)

        Compute a histogram using the provided buckets. The buckets
        are all open to the right except for the last which is closed.
        e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],
        which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1
        and 50 we would have a histogram of 1,0,1.

        If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),
        this can be switched from an O(log n) inseration to O(1) per
        element(where n = # buckets).

        Buckets must be sorted and not contain any duplicates, must be
        at least two elements.

        If `buckets` is a number, it will generates buckets which is
        evenly spaced between the minimum and maximum of the RDD. For
        example, if the min value is 0 and the max is 100, given buckets
        as 2, the resulting buckets will be [0,50) [50,100]. buckets must
        be at least 1 If the RDD contains infinity, NaN throws an exception
        If the elements in RDD do not vary (max == min) always returns
        a single bucket.

        It will return an tuple of buckets and histogram.

        >>> rdd = sc.parallelize(range(51))
        >>> rdd.histogram(2)
        ([0, 25, 50], [25, 26])
        >>> rdd.histogram([0, 5, 25, 50])
        ([0, 5, 25, 50], [5, 20, 26])
        >>> rdd.histogram([0, 15, 30, 45, 60], True)
        ([0, 15, 30, 45, 60], [15, 15, 15, 6])
        >>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"])
        >>> rdd.histogram(("a", "b", "c"))
        (('a', 'b', 'c'), [2, 2])

closes #122, it's duplicated.

Author: Davies Liu <davies.liu@gmail.com>

Closes #2091 from davies/histgram and squashes the following commits:

a322f8a [Davies Liu] fix deprecation of e.message
84e85fa [Davies Liu] remove evenBuckets, add more tests (including str)
d9a0722 [Davies Liu] address comments
0e18a2d [Davies Liu] add histgram() API
2014-08-26 13:04:30 -07:00
Davies Liu fb0db77242 [SPARK-2871] [PySpark] add zipWithIndex() and zipWithUniqueId()
RDD.zipWithIndex()

        Zips this RDD with its element indices.

        The ordering is first based on the partition index and then the
        ordering of items within each partition. So the first item in
        the first partition gets index 0, and the last item in the last
        partition receives the largest index.

        This method needs to trigger a spark job when this RDD contains
        more than one partitions.

        >>> sc.parallelize(range(4), 2).zipWithIndex().collect()
        [(0, 0), (1, 1), (2, 2), (3, 3)]

RDD.zipWithUniqueId()

        Zips this RDD with generated unique Long ids.

        Items in the kth partition will get ids k, n+k, 2*n+k, ..., where
        n is the number of partitions. So there may exist gaps, but this
        method won't trigger a spark job, which is different from
        L{zipWithIndex}

        >>> sc.parallelize(range(4), 2).zipWithUniqueId().collect()
        [(0, 0), (2, 1), (1, 2), (3, 3)]

Author: Davies Liu <davies.liu@gmail.com>

Closes #2092 from davies/zipWith and squashes the following commits:

cebe5bf [Davies Liu] improve test cases, reverse the order of index
0d2a128 [Davies Liu] add zipWithIndex() and zipWithUniqueId()
2014-08-24 21:16:05 -07:00
Davies Liu 8df4dad495 [SPARK-2871] [PySpark] add approx API for RDD
RDD.countApprox(self, timeout, confidence=0.95)

        :: Experimental ::
        Approximate version of count() that returns a potentially incomplete
        result within a timeout, even if not all tasks have finished.

        >>> rdd = sc.parallelize(range(1000), 10)
        >>> rdd.countApprox(1000, 1.0)
        1000

RDD.sumApprox(self, timeout, confidence=0.95)

        Approximate operation to return the sum within a timeout
        or meet the confidence.

        >>> rdd = sc.parallelize(range(1000), 10)
        >>> r = sum(xrange(1000))
        >>> (rdd.sumApprox(1000) - r) / r < 0.05

RDD.meanApprox(self, timeout, confidence=0.95)

        :: Experimental ::
        Approximate operation to return the mean within a timeout
        or meet the confidence.

        >>> rdd = sc.parallelize(range(1000), 10)
        >>> r = sum(xrange(1000)) / 1000.0
        >>> (rdd.meanApprox(1000) - r) / r < 0.05
        True

Author: Davies Liu <davies.liu@gmail.com>

Closes #2095 from davies/approx and squashes the following commits:

e8c252b [Davies Liu] add approx API for RDD
2014-08-23 19:33:34 -07:00
Davies Liu db436e36c4 [SPARK-2871] [PySpark] add key argument for max(), min() and top(n)
RDD.max(key=None)

        param key: A function used to generate key for comparing

        >>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])
        >>> rdd.max()
        43.0
        >>> rdd.max(key=str)
        5.0

RDD.min(key=None)

        Find the minimum item in this RDD.

        param key: A function used to generate key for comparing

        >>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])
        >>> rdd.min()
        2.0
        >>> rdd.min(key=str)
        10.0

RDD.top(num, key=None)

        Get the top N elements from a RDD.

        Note: It returns the list sorted in descending order.
        >>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
        [12]
        >>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2)
        [6, 5]
        >>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)
        [4, 3, 2]

Author: Davies Liu <davies.liu@gmail.com>

Closes #2094 from davies/cmp and squashes the following commits:

ccbaf25 [Davies Liu] add `key` to top()
ad7e374 [Davies Liu] fix tests
2f63512 [Davies Liu] change `comp` to `key` in min/max
dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min()
2014-08-23 18:55:13 -07:00
Davies Liu 0a7ef6339f [SPARK-3141] [PySpark] fix sortByKey() with take()
Fix sortByKey() with take()

The function `f` used in mapPartitions should always return an iterator.

Author: Davies Liu <davies.liu@gmail.com>

Closes #2045 from davies/fix_sortbykey and squashes the following commits:

1160f59 [Davies Liu] fix sortByKey() with take()
2014-08-19 22:43:49 -07:00
Davies Liu d7e80c2597 [SPARK-2790] [PySpark] fix zip with serializers which have different batch sizes.
If two RDDs have different batch size in serializers, then it will try to re-serialize the one with smaller batch size, then call RDD.zip() in Spark.

Author: Davies Liu <davies.liu@gmail.com>

Closes #1894 from davies/zip and squashes the following commits:

c4652ea [Davies Liu] add more test cases
6d05fc8 [Davies Liu] Merge branch 'master' into zip
813b1e4 [Davies Liu] add more tests for failed cases
a4aafda [Davies Liu] fix zip with serializers which have different batch sizes.
2014-08-19 14:46:32 -07:00
Josh Rosen 1f1819b20f [SPARK-3114] [PySpark] Fix Python UDFs in Spark SQL.
This fixes SPARK-3114, an issue where we inadvertently broke Python UDFs in Spark SQL.

This PR modifiers the test runner script to always run the PySpark SQL tests, irrespective of whether SparkSQL itself has been modified.  It also includes Davies' fix for the bug.

Closes #2026.

Author: Josh Rosen <joshrosen@apache.org>
Author: Davies Liu <davies.liu@gmail.com>

Closes #2027 from JoshRosen/pyspark-sql-fix and squashes the following commits:

9af2708 [Davies Liu] bugfix: disable compression of command
0d8d3a4 [Josh Rosen] Always run Python Spark SQL tests.
2014-08-18 20:42:19 -07:00
Davies Liu d1d0ee41c2 [SPARK-3103] [PySpark] fix saveAsTextFile() with utf-8
bugfix: It will raise an exception when it try to encode non-ASCII strings into unicode. It should only encode unicode as "utf-8".

Author: Davies Liu <davies.liu@gmail.com>

Closes #2018 from davies/fix_utf8 and squashes the following commits:

4db7967 [Davies Liu] fix saveAsTextFile() with utf-8
2014-08-18 13:58:35 -07:00
Davies Liu 2fc8aca086 [SPARK-1065] [PySpark] improve supporting for large broadcast
Passing large object by py4j is very slow (cost much memory), so pass broadcast objects via files (similar to parallelize()).

Add an option to keep object in driver (it's False by default) to save memory in driver.

Author: Davies Liu <davies.liu@gmail.com>

Closes #1912 from davies/broadcast and squashes the following commits:

e06df4a [Davies Liu] load broadcast from disk in driver automatically
db3f232 [Davies Liu] fix serialization of accumulator
631a827 [Davies Liu] Merge branch 'master' into broadcast
c7baa8c [Davies Liu] compress serrialized broadcast and command
9a7161f [Davies Liu] fix doc tests
e93cf4b [Davies Liu] address comments: add test
6226189 [Davies Liu] improve large broadcast
2014-08-16 16:59:34 -07:00
Davies Liu 434bea1c00 [SPARK-2983] [PySpark] improve performance of sortByKey()
1. skip partitionBy() when numOfPartition is 1
2. use bisect_left (O(lg(N))) instread of loop (O(N)) in
rangePartitioner

Author: Davies Liu <davies.liu@gmail.com>

Closes #1898 from davies/sort and squashes the following commits:

0a9608b [Davies Liu] Merge branch 'master' into sort
1cf9565 [Davies Liu] improve performance of sortByKey()
2014-08-13 14:57:12 -07:00
RJ Nowling e537b33c63 [PySpark] Add blanklines to Python docstrings so example code renders correctly
Author: RJ Nowling <rnowling@gmail.com>

Closes #1808 from rnowling/pyspark_docs and squashes the following commits:

c06d774 [RJ Nowling] Add blanklines to Python docstrings so example code renders correctly
2014-08-06 14:12:21 -07:00
Nicholas Chammas d614967b0b [SPARK-2627] [PySpark] have the build enforce PEP 8 automatically
As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that.

Notes:
* We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server.
* I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request.
* I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete.
* Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo.

Author: Nicholas Chammas <nicholas.chammas@gmail.com>
Author: nchammas <nicholas.chammas@gmail.com>

Closes #1744 from nchammas/master and squashes the following commits:

274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes
983d963 [nchammas] Merge pull request #5 from apache/master
1db5314 [nchammas] Merge pull request #4 from apache/master
0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes
bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing
6db9a44 [nchammas] Merge pull request #3 from apache/master
7b4750e [Nicholas Chammas] merge upstream changes
91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks
44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files
b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily
bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes
9da347f [nchammas] Merge pull request #2 from apache/master
aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks
d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine
dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime
a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections
21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8
6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes
fe57ed0 [Nicholas Chammas] removing merge conflict backups
9c01d4c [nchammas] Merge pull request #1 from apache/master
9a66cb0 [Nicholas Chammas] resolving merge conflicts
a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes
beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status
723ed39 [Nicholas Chammas] always delete the report file
0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests
12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter
61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter
75ad552 [Nicholas Chammas] make check output style consistent
2014-08-06 12:58:24 -07:00
Davies Liu 880eabec37 [SPARK-2010] [PySpark] [SQL] support nested structure in SchemaRDD
Convert Row in JavaSchemaRDD into Array[Any] and unpickle them as tuple in Python, then convert them into namedtuple, so use can access fields just like attributes.

This will let nested structure can be accessed as object, also it will reduce the size of serialized data and better performance.

root
 |-- field1: integer (nullable = true)
 |-- field2: string (nullable = true)
 |-- field3: struct (nullable = true)
 |    |-- field4: integer (nullable = true)
 |    |-- field5: array (nullable = true)
 |    |    |-- element: integer (containsNull = false)
 |-- field6: array (nullable = true)
 |    |-- element: struct (containsNull = false)
 |    |    |-- field7: string (nullable = true)

Then we can access them by row.field3.field5[0]  or row.field6[5].field7

It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType.

You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as:

ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1]))

Or you could use Row to create a class just like namedtuple, for example:

Person = Row("name", "age")
ctx.inferSchema(rdd.map(lambda x: Person(*x)))

Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details.

schema = StructType([StructField("name, StringType, True),
                                    StructType("age", IntegerType, True)])
ctx.applySchema(rdd, schema)

PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable.

Author: Davies Liu <davies.liu@gmail.com>

Closes #1598 from davies/nested and squashes the following commits:

f1d15b6 [Davies Liu] verify schema with the first few rows
8852aaf [Davies Liu] check type of schema
abe9e6e [Davies Liu] address comments
61b2292 [Davies Liu] add @deprecated to pythonToJavaMap
1e5b801 [Davies Liu] improve cache of classes
51aa135 [Davies Liu] use Row to infer schema
e9c0d5c [Davies Liu] remove string typed schema
353a3f2 [Davies Liu] fix code style
63de8f8 [Davies Liu] fix typo
c79ca67 [Davies Liu] fix serialization of nested data
6b258b5 [Davies Liu] fix pep8
9d8447c [Davies Liu] apply schema provided by string of names
f5df97f [Davies Liu] refactor, address comments
9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python
84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested
0eaaf56 [Davies Liu] fix doc tests
b3559b4 [Davies Liu] use generated Row instead of namedtuple
c4ddc30 [Davies Liu] fix conflict between name of fields and variables
7f6f251 [Davies Liu] address all comments
d69d397 [Davies Liu] refactor
2cc2d45 [Davies Liu] refactor
182fb46 [Davies Liu] refactor
bc6e9e1 [Davies Liu] switch to new Schema API
547bf3e [Davies Liu] Merge branch 'master' into nested
a435b5a [Davies Liu] add docs and code refactor
2c8debc [Davies Liu] Merge branch 'master' into nested
644665a [Davies Liu] use tuple and namedtuple for schemardd
2014-08-01 18:47:41 -07:00
Kan Zhang 94d1f46fc4 [SPARK-2024] Add saveAsSequenceFile to PySpark
JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024

This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats.

* Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs.

* Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types.

* No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples.

* Added HBase and Cassandra output examples to show how custom output formats and converters can be used.

cc MLnick mateiz ahirreddy pwendell

Author: Kan Zhang <kzhang@apache.org>

Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits:

c01e3ef [Kan Zhang] [SPARK-2024] code formatting
6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD
d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10
57a7a5e [Kan Zhang] [SPARK-2024] correcting typo
75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD
0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests
9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests
0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases
7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
2014-07-30 13:19:05 -07:00
Josh Rosen ba46bbed5d [SPARK-2601] [PySpark] Fix Py4J error when transforming pickleFiles
Similar to SPARK-1034, the problem was that Py4J didn’t cope well with the fake ClassTags used in the Java API.  It doesn’t look like there’s any reason why PythonRDD needs to take a ClassTag, since it just ignores the type of the previous RDD, so I removed the type parameter and we no longer pass ClassTags from Python.

Author: Josh Rosen <joshrosen@apache.org>

Closes #1605 from JoshRosen/spark-2601 and squashes the following commits:

b68e118 [Josh Rosen] Fix Py4J error when transforming pickleFiles [SPARK-2601]
2014-07-26 17:37:05 -07:00
Doris Xin 2f75a4a30e [SPARK-2656] Python version of stratified sampling
exact sample size not supported for now.

Author: Doris Xin <doris.s.xin@gmail.com>

Closes #1554 from dorx/pystratified and squashes the following commits:

4ba927a [Doris Xin] use rel diff (+- 50%) instead of abs diff (+- 50)
bdc3f8b [Doris Xin] updated unit to check sample holistically
7713c7b [Doris Xin] Python version of stratified sampling
2014-07-24 23:42:08 -07:00
Davies Liu 14174abd42 [SPARK-2538] [PySpark] Hash based disk spilling aggregation
During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation.

It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition).

Author: Davies Liu <davies.liu@gmail.com>

Closes #1460 from davies/spill and squashes the following commits:

cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible.
37d71f7 [Davies Liu] balance the partitions
902f036 [Davies Liu] add shuffle.py into run-tests
dcf03a9 [Davies Liu] fix memory_info() of psutil
67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS
f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge:
e74b785 [Davies Liu] fix code style and change next_limit to memory_limit
400be01 [Davies Liu] address all the comments
6178844 [Davies Liu] refactor and improve docs
fdd0a49 [Davies Liu] add long doc string for ExternalMerger
1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy()
e6cc7f9 [Davies Liu] Merge branch 'master' into spill
3652583 [Davies Liu] address comments
e78a0a0 [Davies Liu] fix style
24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR
57ee7ef [Davies Liu] update docs
286aaff [Davies Liu] let spilled aggregation in Python configurable
e9a40f6 [Davies Liu] recursive merger
6edbd1f [Davies Liu] Hash based disk spilling aggregation
2014-07-24 22:53:47 -07:00
Prashant Sharma eff9714e1c [SPARK-2014] Make PySpark store RDDs in MEMORY_ONLY_SER with compression by default
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #1051 from ScrapCodes/SPARK-2014/pyspark-cache and squashes the following commits:

f192df7 [Prashant Sharma] Code Review
2a2f43f [Prashant Sharma] [SPARK-2014] Make PySpark store RDDs in MEMORY_ONLY_SER with compression by default
2014-07-24 18:15:37 -07:00
Davies Liu 872538c600 [SPARK-2494] [PySpark] make hash of None consistant cross machines
In CPython, hash of None is different cross machines, it will cause wrong result during shuffle. This PR will fix this.

Author: Davies Liu <davies.liu@gmail.com>

Closes #1371 from davies/hash_of_none and squashes the following commits:

d01745f [Davies Liu] add comments, remove outdated unit tests
5467141 [Davies Liu] disable hijack of hash, use it only for partitionBy()
b7118aa [Davies Liu] use __builtin__ instead of __builtins__
839e417 [Davies Liu] hijack hash to make hash of None consistant cross machines
2014-07-21 11:59:54 -07:00
Prashant Sharma aab5349660 Made rdd.py pep8 complaint by using Autopep8 and a little manual editing.
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #1354 from ScrapCodes/pep8-comp-1 and squashes the following commits:

9858ea8 [Prashant Sharma] Code Review
d8851b7 [Prashant Sharma] Found # noqa works even inside comment blocks. Not sure if it works with all versions of python.
10c0cef [Prashant Sharma] Made rdd.py pep8 complaint by using Autopep8 and a little manual tweaking.
2014-07-14 00:42:59 -07:00
Anant 010c460d62 [SPARK-2061] Made splits deprecated in JavaRDDLike
The jira for the issue can be found at: https://issues.apache.org/jira/browse/SPARK-2061
Most of spark has used over to consistently using `partitions` instead of `splits`. We should do likewise and add a `partitions` method to JavaRDDLike and have `splits` just call that. We should also go through all cases where other API's (e.g. Python) call `splits` and we should change those to use the newer API.

Author: Anant <anant.asty@gmail.com>

Closes #1062 from anantasty/SPARK-2061 and squashes the following commits:

b83ce6b [Anant] Fixed syntax issue
21f9210 [Anant] Fixed version number in deprecation string
9315b76 [Anant] made related changes to use partitions in python api
8c62dd1 [Anant] Made splits deprecated in JavaRDDLike
2014-06-20 18:57:24 -07:00
Allan Douglas R. de Oliveira 6a224c31e8 SPARK-1868: Users should be allowed to cogroup at least 4 RDDs
Adds cogroup for 4 RDDs.

Author: Allan Douglas R. de Oliveira <allandouglas@gmail.com>

Closes #813 from douglaz/more_cogroups and squashes the following commits:

f8d6273 [Allan Douglas R. de Oliveira] Test python groupWith for one more case
0e9009c [Allan Douglas R. de Oliveira] Added scala tests
c3ffcdd [Allan Douglas R. de Oliveira] Added java tests
517a67f [Allan Douglas R. de Oliveira] Added tests for python groupWith
2f402d5 [Allan Douglas R. de Oliveira] Removed TODO
17474f4 [Allan Douglas R. de Oliveira] Use new cogroup function
7877a2a [Allan Douglas R. de Oliveira] Fixed code
ba02414 [Allan Douglas R. de Oliveira] Added varargs cogroup to pyspark
c4a8a51 [Allan Douglas R. de Oliveira] Added java cogroup 4
e94963c [Allan Douglas R. de Oliveira] Fixed spacing
f1ee57b [Allan Douglas R. de Oliveira] Fixed scala style issues
d7196f1 [Allan Douglas R. de Oliveira] Allow the cogroup of 4 RDDs
2014-06-20 11:03:03 -07:00
Aaron Davidson f46e02fcdb SPARK-2203: PySpark defaults to use same num reduce partitions as map side
For shuffle-based operators, such as rdd.groupBy() or rdd.sortByKey(), PySpark will always assume that the default parallelism to use for the reduce side is ctx.defaultParallelism, which is a constant typically determined by the number of cores in cluster.

In contrast, Spark's Partitioner#defaultPartitioner will use the same number of reduce partitions as map partitions unless the defaultParallelism config is explicitly set. This tends to be a better default in order to avoid OOMs, and should also be the behavior of PySpark.

JIRA: https://issues.apache.org/jira/browse/SPARK-2203

Author: Aaron Davidson <aaron@databricks.com>

Closes #1138 from aarondav/pyfix and squashes the following commits:

1bd5751 [Aaron Davidson] SPARK-2203: PySpark defaults to use same num reduce partitions as map partitions
2014-06-20 00:06:57 -07:00
Sandy Ryza 2794990e9e SPARK-2146. Fix takeOrdered doc
Removes Python syntax in Scaladoc, corrects result in Scaladoc, and removes irrelevant cache() call in Python doc.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #1086 from sryza/sandy-spark-2146 and squashes the following commits:

185ff18 [Sandy Ryza] Use Seq instead of Array
c996120 [Sandy Ryza] SPARK-2146.  Fix takeOrdered doc
2014-06-17 12:03:22 -07:00
Andrew Ash b92d16b114 SPARK-1063 Add .sortBy(f) method on RDD
This never got merged from the apache/incubator-spark repo (which is now deleted) but there had been several rounds of code review on this PR there.

I think this is ready for merging.

Author: Andrew Ash <andrew@andrewash.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@apache.org>

Closes #369 from ash211/sortby and squashes the following commits:

d09147a [Andrew Ash] Fix Ordering import
43d0a53 [Andrew Ash] Fix missing .collect()
29a54ed [Andrew Ash] Re-enable test by converting to a closure
5a95348 [Andrew Ash] Add license for RDDSuiteUtils
64ed6e3 [Andrew Ash] Remove leaked diff
d4de69a [Andrew Ash] Remove scar tissue
63638b5 [Andrew Ash] Add Python version of .sortBy()
45e0fde [Andrew Ash] Add Java version of .sortBy()
adf84c5 [Andrew Ash] Re-indent to keep line lengths under 100 chars
9d9b9d8 [Andrew Ash] Use parentheses on .collect() calls
0457b69 [Andrew Ash] Ignore failing test
99f0baf [Andrew Ash] Merge branch 'master' into sortby
222ae97 [Andrew Ash] Try moving Ordering objects out to a different class
3fd0dd3 [Andrew Ash] Add (failing) test for sortByKey with explicit Ordering
b8b5bbc [Andrew Ash] Align remove extra spaces that were used to align ='s in test code
8c53298 [Andrew Ash] Actually use ascending and numPartitions parameters
381eef2 [Andrew Ash] Correct silly typo
7db3e84 [Andrew Ash] Support ascending and numPartitions params in sortBy()
0f685fd [Andrew Ash] Merge remote-tracking branch 'origin/master' into sortby
ca4490d [Andrew Ash] Add .sortBy(f) method on RDD
2014-06-17 11:47:48 -07:00
Kan Zhang d81c08bac9 [SPARK-2130] End-user friendly String repr for StorageLevel in Python
JIRA issue https://issues.apache.org/jira/browse/SPARK-2130

This PR adds an end-user friendly String representation for StorageLevel
in Python, similar to ```StorageLevel.description``` in Scala.
```
>>> rdd = sc.parallelize([1,2])
>>> storage_level = rdd.getStorageLevel()
>>> storage_level
StorageLevel(False, False, False, False, 1)
>>> print(storage_level)
Serialized 1x Replicated
```

Author: Kan Zhang <kzhang@apache.org>

Closes #1096 from kanzhang/SPARK-2130 and squashes the following commits:

7c8b98b [Kan Zhang] [SPARK-2130] Prettier epydoc output
cc5bf45 [Kan Zhang] [SPARK-2130] End-user friendly String representation for StorageLevel in Python
2014-06-16 23:31:31 -07:00
Doris Xin 1de1d703bf SPARK-1939 Refactor takeSample method in RDD to use ScaSRS
Modified the takeSample method in RDD to use the ScaSRS sampling technique to improve performance. Added a private method that computes sampling rate > sample_size/total to ensure sufficient sample size with success rate >= 0.9999. Added a unit test for the private method to validate choice of sampling rate.

Author: Doris Xin <doris.s.xin@gmail.com>
Author: dorx <doris.s.xin@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #916 from dorx/takeSample and squashes the following commits:

5b061ae [Doris Xin] merge master
444e750 [Doris Xin] edge cases
3de882b [dorx] Merge pull request #2 from mengxr/SPARK-1939
82dde31 [Xiangrui Meng] update pyspark's takeSample
48d954d [Doris Xin] remove unused imports from RDDSuite
fb1452f [Doris Xin] allowing num to be greater than count in all cases
1481b01 [Doris Xin] washing test tubes and making coffee
dc699f3 [Doris Xin] give back imports removed by accident in rdd.py
64e445b [Doris Xin] logwarnning as soon as it enters the while loop
55518ed [Doris Xin] added TODO for logging in rdd.py
eff89e2 [Doris Xin] addressed reviewer comments.
ecab508 [Doris Xin] "fixed checkstyle violation
0a9b3e3 [Doris Xin] "reviewer comment addressed"
f80f270 [Doris Xin] Merge branch 'master' into takeSample
ae3ad04 [Doris Xin] fixed edge cases to prevent overflow
065ebcd [Doris Xin] Merge branch 'master' into takeSample
9bdd36e [Doris Xin] Check sample size and move computeFraction
e3fd6a6 [Doris Xin] Merge branch 'master' into takeSample
7cab53a [Doris Xin] fixed import bug in rdd.py
ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD
1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS
2014-06-12 19:44:27 -07:00
Sandy Ryza ce92a9c18f SPARK-554. Add aggregateByKey.
Author: Sandy Ryza <sandy@cloudera.com>

Closes #705 from sryza/sandy-spark-554 and squashes the following commits:

2302b8f [Sandy Ryza] Add MIMA exclude
f52e0ad [Sandy Ryza] Fix Python tests for real
2f3afa3 [Sandy Ryza] Fix Python test
0b735e9 [Sandy Ryza] Fix line lengths
ae56746 [Sandy Ryza] Fix doc (replace T with V)
c2be415 [Sandy Ryza] Java and Python aggregateByKey
23bf400 [Sandy Ryza] SPARK-554.  Add aggregateByKey.
2014-06-12 08:14:25 -07:00
Jeff Thompson 43d53d51c9 fixed typo in docstring for min()
Hi, I found this typo while learning spark and thought I'd do a pull request.

Author: Jeff Thompson <jeffreykeatingthompson@gmail.com>

Closes #1065 from jkthompson/docstring-typo-minmax and squashes the following commits:

29b6a26 [Jeff Thompson] fixed typo in docstring for min()
2014-06-12 08:10:51 -07:00
Syed Hashmi 6113ac1559 [SPARK-1308] Add getNumPartitions to pyspark RDD
Add getNumPartitions to pyspark RDD to provide an intuitive way to get number of partitions in RDD like we can do in scala today.

Author: Syed Hashmi <shashmi@cloudera.com>

Closes #995 from syedhashmi/master and squashes the following commits:

de0ed5e [Syed Hashmi] [SPARK-1308] Add getNumPartitions to pyspark RDD
2014-06-09 00:08:40 -07:00
Kan Zhang 21e40ed88b [SPARK-1161] Add saveAsPickleFile and SparkContext.pickleFile in Python
Author: Kan Zhang <kzhang@apache.org>

Closes #755 from kanzhang/SPARK-1161 and squashes the following commits:

24ed8a2 [Kan Zhang] [SPARK-1161] Fixing doc tests
44e0615 [Kan Zhang] [SPARK-1161] Adding an optional batchSize with default value 10
d929429 [Kan Zhang] [SPARK-1161] Add saveAsObjectFile and SparkContext.objectFile in Python
2014-06-03 18:18:25 -07:00
Erik Selin 8edc9d0330 [SPARK-1468] Modify the partition function used by partitionBy.
Make partitionBy use a tweaked version of hash as its default partition function
since the python hash function does not consistently assign the same value
to None across python processes.

Associated JIRA at https://issues.apache.org/jira/browse/SPARK-1468

Author: Erik Selin <erik.selin@jadedpixel.com>

Closes #371 from tyro89/consistent_hashing and squashes the following commits:

201c301 [Erik Selin] Make partitionBy use a tweaked version of hash as its default partition function since the python hash function does not consistently assign the same value to None across python processes.
2014-06-03 13:31:16 -07:00
Aaron Davidson 9909efc10a SPARK-1839: PySpark RDD#take() shouldn't always read from driver
This patch simply ports over the Scala implementation of RDD#take(), which reads the first partition at the driver, then decides how many more partitions it needs to read and will possibly start a real job if it's more than 1. (Note that SparkContext#runJob(allowLocal=true) only runs the job locally if there's 1 partition selected and no parent stages.)

Author: Aaron Davidson <aaron@databricks.com>

Closes #922 from aarondav/take and squashes the following commits:

fa06df9 [Aaron Davidson] SPARK-1839: PySpark RDD#take() shouldn't always read from driver
2014-05-31 13:04:57 -07:00
Patrick Wendell 21570b4633 Documentation: Encourage use of reduceByKey instead of groupByKey.
Author: Patrick Wendell <pwendell@gmail.com>

Closes #784 from pwendell/group-by-key and squashes the following commits:

9b4505f [Patrick Wendell] Small fix
6347924 [Patrick Wendell] Documentation: Encourage use of reduceByKey instead of groupByKey.
2014-05-14 22:24:04 -07:00
Kan Zhang 6c2691d0a0 [SPARK-1690] Tolerating empty elements when saving Python RDD to text files
Tolerate empty strings in PythonRDD

Author: Kan Zhang <kzhang@apache.org>

Closes #644 from kanzhang/SPARK-1690 and squashes the following commits:

c62ad33 [Kan Zhang] Adding Python doctest
473ec4b [Kan Zhang] [SPARK-1690] Tolerating empty elements when saving Python RDD to text files
2014-05-10 14:01:08 -07:00
Xiangrui Meng d33df1c151 [SPARK-1674] fix interrupted system call error in pyspark's RDD.pipe
`RDD.pipe`'s doctest throws interrupted system call exception on Mac. It can be fixed by wrapping `pipe.stdout.readline` in an iterator.

Author: Xiangrui Meng <meng@databricks.com>

Closes #594 from mengxr/pyspark-pipe and squashes the following commits:

cc32ac9 [Xiangrui Meng] fix interrupted system call error in pyspark's RDD.pipe
2014-04-29 18:06:45 -07:00
Holden Karau e03bc379ee SPARK-1242 Add aggregate to python rdd
Author: Holden Karau <holden@pigscanfly.ca>

Closes #139 from holdenk/add_aggregate_to_python_api and squashes the following commits:

0f39ae3 [Holden Karau] Merge in master
4879c75 [Holden Karau] CR feedback, fix issue with empty RDDs in aggregate
70b4724 [Holden Karau] Style fixes from code review
96b047b [Holden Karau] Add aggregate to python rdd
2014-04-24 23:07:54 -07:00
Arun Ramakrishnan 35e3d199f0 SPARK-1438 RDD.sample() make seed param optional
copying form previous pull request https://github.com/apache/spark/pull/462

Its probably better to let the underlying language implementation take care of the default . This was easier to do with python as the default value for seed in random and numpy random is None.

In Scala/Java side it might mean propagating an Option or null(oh no!) down the chain until where the Random is constructed. But, looks like the convention in some other methods was to use System.nanoTime. So, followed that convention.

Conflict with overloaded method in sql.SchemaRDD.sample which also defines default params.
sample(fraction, withReplacement=false, seed=math.random)
Scala does not allow more than one overloaded to have default params. I believe the author intended to override the RDD.sample method and not overload it. So, changed it.

If backward compatible is important, 3 new method can be introduced (without default params) like this
sample(fraction)
sample(fraction, withReplacement)
sample(fraction, withReplacement, seed)

Added some tests for the scala RDD takeSample method.

Author: Arun Ramakrishnan <smartnut007@gmail.com>

This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>

Closes #477 from smartnut007/master and squashes the following commits:

07bb06e [Arun Ramakrishnan] SPARK-1438 fixing more space formatting issues
b9ebfe2 [Arun Ramakrishnan] SPARK-1438 removing redundant import of random in python rddsampler
8d05b1a [Arun Ramakrishnan] SPARK-1438 RDD . Replace System.nanoTime with a Random generated number. python: use a separate instance of Random instead of seeding language api global Random instance.
69619c6 [Arun Ramakrishnan] SPARK-1438 fix spacing issue
0c247db [Arun Ramakrishnan] SPARK-1438 RDD language apis to support optional seed in RDD methods sample/takeSample
2014-04-24 17:27:16 -07:00
Holden Karau ce8ec54561 Spark 1271: Co-Group and Group-By should pass Iterable[X]
Author: Holden Karau <holden@pigscanfly.ca>

Closes #242 from holdenk/spark-1320-cogroupandgroupshouldpassiterator and squashes the following commits:

f289536 [Holden Karau] Fix bad merge, should have been Iterable rather than Iterator
77048f8 [Holden Karau] Fix merge up to master
d3fe909 [Holden Karau] use toSeq instead
7a092a3 [Holden Karau] switch resultitr to resultiterable
eb06216 [Holden Karau] maybe I should have had a coffee first. use correct import for guava iterables
c5075aa [Holden Karau] If guava 14 had iterables
2d06e10 [Holden Karau] Fix Java 8 cogroup tests for the new API
11e730c [Holden Karau] Fix streaming tests
66b583d [Holden Karau] Fix the core test suite to compile
4ed579b [Holden Karau] Refactor from iterator to iterable
d052c07 [Holden Karau] Python tests now pass with iterator pandas
3bcd81d [Holden Karau] Revert "Try and make pickling list iterators work"
cd1e81c [Holden Karau] Try and make pickling list iterators work
c60233a [Holden Karau] Start investigating moving to iterators for python API like the Java/Scala one. tl;dr: We will have to write our own iterator since the default one doesn't pickle well
88a5cef [Holden Karau] Fix cogroup test in JavaAPISuite for streaming
a5ee714 [Holden Karau] oops, was checking wrong iterator
e687f21 [Holden Karau] Fix groupbykey test in JavaAPISuite of streaming
ec8cc3e [Holden Karau] Fix test issues\!
4b0eeb9 [Holden Karau] Switch cast in PairDStreamFunctions
fa395c9 [Holden Karau] Revert "Add a join based on the problem in SVD"
ec99e32 [Holden Karau] Revert "Revert this but for now put things in list pandas"
b692868 [Holden Karau] Revert
7e533f7 [Holden Karau] Fix the bug
8a5153a [Holden Karau] Revert me, but we have some stuff to debug
b4e86a9 [Holden Karau] Add a join based on the problem in SVD
c4510e2 [Holden Karau] Revert this but for now put things in list pandas
b4e0b1d [Holden Karau] Fix style issues
71e8b9f [Holden Karau] I really need to stop calling size on iterators, it is the path of sadness.
b1ae51a [Holden Karau] Fix some of the types in the streaming JavaAPI suite. Probably still needs more work
37888ec [Holden Karau] core/tests now pass
249abde [Holden Karau] org.apache.spark.rdd.PairRDDFunctionsSuite passes
6698186 [Holden Karau] Revert "I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy"
fe992fe [Holden Karau] hmmm try and fix up basic operation suite
172705c [Holden Karau] Fix Java API suite
caafa63 [Holden Karau] I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy
88b3329 [Holden Karau] Fix groupbykey to actually give back an iterator
4991af6 [Holden Karau] Fix some tests
be50246 [Holden Karau] Calling size on an iterator is not so good if we want to use it after
687ffbc [Holden Karau] This is the it compiles point of replacing Seq with Iterator and JList with JIterator in the groupby and cogroup signatures
2014-04-08 18:15:59 -07:00
Haoyuan Li b50ddfde03 SPARK-1305: Support persisting RDD's directly to Tachyon
Move the PR#468 of apache-incubator-spark to the apache-spark
"Adding an option to persist Spark RDD blocks into Tachyon."

Author: Haoyuan Li <haoyuan@cs.berkeley.edu>
Author: RongGu <gurongwalker@gmail.com>

Closes #158 from RongGu/master and squashes the following commits:

72b7768 [Haoyuan Li] merge master
9f7fa1b [Haoyuan Li] fix code style
ae7834b [Haoyuan Li] minor cleanup
a8b3ec6 [Haoyuan Li] merge master branch
e0f4891 [Haoyuan Li] better check offheap.
55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel
7cd4600 [RongGu] remove some logic code for tachyonstore's replication
51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore
8adfcfa [RongGu] address arron's comment on inTachyonSize
120e48a [RongGu] changed the root-level dir name in Tachyon
5cc041c [Haoyuan Li] address aaron's comments
9b97935 [Haoyuan Li] address aaron's comments
d9a6438 [Haoyuan Li] fix for pspark
77d2703 [Haoyuan Li] change python api.git status
3dcace4 [Haoyuan Li] address matei's comments
91fa09d [Haoyuan Li] address patrick's comments
589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE
64348b2 [Haoyuan Li] update conf docs.
ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1
619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore
be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler
49cc724 [Haoyuan Li] update docs with off_headp option
4572f9f [RongGu] reserving the old apply function API of StorageLevel
04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP
c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP
76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md
e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments
fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix
939e467 [Haoyuan Li] 0.4.1-thrift from maven central
86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1
16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift
eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1
bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem
6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1
d827250 [RongGu] fix JsonProtocolSuie test failure
716e93b [Haoyuan Li] revert the version
ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift
2825a13 [RongGu] up-merging to the current master branch of the apache spark
6a22c1a [Haoyuan Li] fix scalastyle
8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client.
77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice.
1dcadf9 [Haoyuan Li] typo
bf278fa [Haoyuan Li] fix python tests
e82909c [Haoyuan Li] minor cleanup
776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR
8859371 [Haoyuan Li] various minor fixes and clean up
e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode.
fcaeab2 [Haoyuan Li] address Aaron's comment
e554b1e [Haoyuan Li] add python code
47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels.
dc8ef24 [Haoyuan Li] add old storelevel constructor
e01a271 [Haoyuan Li] update tachyon 0.4.1
8011a96 [RongGu] fix a brought-in mistake in StorageLevel
70ca182 [RongGu] a bit change in comment
556978b [RongGu] fix the scalastyle errors
791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 20:38:20 -07:00
Prashant Sharma c1ea3afb51 Spark 1162 Implemented takeOrdered in pyspark.
Since python does not have a library for max heap and usual tricks like inverting values etc.. does not work for all cases.

We have our own implementation of max heap.

Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #97 from ScrapCodes/SPARK-1162/pyspark-top-takeOrdered2 and squashes the following commits:

35f86ba [Prashant Sharma] code review
2b1124d [Prashant Sharma] fixed tests
e8a08e2 [Prashant Sharma] Code review comments.
49e6ba7 [Prashant Sharma] SPARK-1162 added takeOrdered to pyspark
2014-04-03 15:42:17 -07:00
Prashant Sharma a0853a39e1 SPARK-1322, top in pyspark should sort result in descending order.
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #235 from ScrapCodes/SPARK-1322/top-rev-sort and squashes the following commits:

f316266 [Prashant Sharma] Minor change in comment.
58e58c6 [Prashant Sharma] SPARK-1322, top in pyspark should sort result in descending order.
2014-03-26 09:16:37 -07:00
Jyotiska NK 67fa71cba2 Added doctest for map function in rdd.py
Doctest added for map in rdd.py

Author: Jyotiska NK <jyotiska123@gmail.com>

Closes #177 from jyotiska/pyspark_rdd_map_doctest and squashes the following commits:

a38527f [Jyotiska NK] Added doctest for map function in rdd.py
2014-03-19 14:04:45 -07:00
Dan McClary e3681f26fa Spark 1246 add min max to stat counter
Here's the addition of min and max to statscounter.py and min and max methods to rdd.py.

Author: Dan McClary <dan.mcclary@gmail.com>

Closes #144 from dwmclary/SPARK-1246-add-min-max-to-stat-counter and squashes the following commits:

fd3fd4b [Dan McClary] fixed  error, updated test
82cde0e [Dan McClary] flipped incorrectly assigned inf values in StatCounter
5d96799 [Dan McClary] added max and min to StatCounter repr for pyspark
21dd366 [Dan McClary] added max and min to StatCounter output, updated doc
1a97558 [Dan McClary] added max and min to StatCounter output, updated doc
a5c13b0 [Dan McClary] Added min and max to Scala and Java RDD, added min and max to StatCounter
ed67136 [Dan McClary] broke min/max out into separate transaction, added to rdd.py
1e7056d [Dan McClary] added underscore to getBucket
37a7dea [Dan McClary] cleaned up boundaries for histogram -- uses real min/max when buckets are derived
29981f2 [Dan McClary] fixed indentation on doctest comment
eaf89d9 [Dan McClary] added correct doctest for histogram
4916016 [Dan McClary] added histogram method, added max and min to statscounter
2014-03-18 00:45:47 -07:00