[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-30 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50577245 QA results for PR 1338:br- This patch FAILED unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-30 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15601428 --- Diff: core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala --- @@ -65,20 +66,49 @@ private[python] object SerDeUtil extends Logging {

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-30 Thread JoshRosen
Github user JoshRosen commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50656448 I think we should remove the `batchSerialized` arguments from PythonRDD's `saveAs*` methods and add a `batchSerialized` field to PythonRDD's constructor, since it's an

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-30 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15601800 --- Diff: core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala --- @@ -65,20 +66,52 @@ private[python] object SerDeUtil extends Logging {

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-30 Thread kanzhang
Github user kanzhang commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15607176 --- Diff: core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala --- @@ -65,20 +66,52 @@ private[python] object SerDeUtil extends Logging {

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-30 Thread JoshRosen
Github user JoshRosen commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50672937 Ah, I see. I don't mind deferring that refactoring to a later patch. I'll create some PySpark refactoring JIRAs later. --- If your project is set up for it, you can

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-30 Thread JoshRosen
Github user JoshRosen commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50673853 I've merged this. Thanks! --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50442216 Looks good to me as well. @JoshRosen any comments? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15508664 --- Diff: core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala --- @@ -65,20 +66,49 @@ private[python] object SerDeUtil extends Logging {

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50544017 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17384/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50544121 QA results for PR 1338:br- This patch FAILED unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread kanzhang
Github user kanzhang commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r1021 --- Diff: core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala --- @@ -65,20 +66,49 @@ private[python] object SerDeUtil extends Logging {

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50545354 Now I have got the following error, since ```saveAsHadoopFile``` has 11 params. Can relax it a bit? ``` Scalastyle checks failed at following occurrences:

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50547331 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17386/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50547384 Nevermind. I'm refactoring. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50547420 QA results for PR 1338:br- This patch FAILED unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50551599 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17389/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50555479 QA results for PR 1338:br- This patch PASSES unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50574875 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17419/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-29 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50576604 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17424/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-28 Thread MLnick
Github user MLnick commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50313472 This looks good to me - I'm +1 to merge. @JoshRosen @mateiz any further comments? Thanks @kanzhang for all the work! --- If your project is set up for

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-28 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50378599 For completeness, added one ```saveAsHadoopDataset()``` and one ```saveAsNewAPIHadoopDataset()``` test. --- If your project is set up for it, you can reply to this

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-28 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50379059 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17292/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-28 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50386062 QA results for PR 1338:br- This patch PASSES unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-28 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50387946 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17301/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-28 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50394525 QA results for PR 1338:br- This patch PASSES unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-26 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50225130 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17218/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-26 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50238113 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17224/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-26 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50240323 QA results for PR 1338:br- This patch PASSES unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50111889 @kanzhang it might mean that we're reusing the Bean object on the Java side when we read from the InputFormat. Hadoop's RecordReaders actually reuse the same object as

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50111934 The same thing happens in normal Spark if you create a hadoopRDD or sequenceFile with Writables inside it, and then call cache(). There will be only one key element and

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15387719 --- Diff: python/pyspark/rdd.py --- @@ -964,6 +964,106 @@ def first(self): return self.take(1)[0] +def

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50112244 BTW you can try WritableUtils.clone. At some point we tried cloning data by default in hadoopRDD, or having a flag for it, and we gave up because it didn't seem to work

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread ash211
Github user ash211 commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50112894 Matei, what InputFormats did you have problems with when cloning by default? I'd love to figure out what it would take to solve the one element/one value problem.

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread MLnick
Github user MLnick commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15388452 --- Diff: python/pyspark/rdd.py --- @@ -964,6 +964,106 @@ def first(self): return self.take(1)[0] +def

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50116221 @ash211 this was the JIRA: https://issues.apache.org/jira/browse/SPARK-1018. I don't remember the problematic types exactly, but one might've been Avro records, so try

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread mateiz
Github user mateiz commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15388927 --- Diff: python/pyspark/rdd.py --- @@ -964,6 +964,106 @@ def first(self): return self.take(1)[0] +def

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50183228 it might mean that we're reusing the Bean object on the Java side when we read from the InputFormat. Hadoop's RecordReaders actually reuse the same object as you read

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread JoshRosen
Github user JoshRosen commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50186945 The batch size is a performance-tuning knob that controls the granularity of data transfer between Python and Java. Larger batch sizes help to amortize certain costs

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread JoshRosen
Github user JoshRosen commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50187471 One difference between this case and `saveAsPickleFile` is that here the actual on-disk output is independent of the batch size, since the data gets unbatched in Java,

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50197882 The latest update fixed JevaBean test failure. To use ```WritableUtils.clone```, I broadcasted Hadoop conf the second time. Is there a way to share the one used by

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50198112 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17195/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50202364 Just wanted to make sure my understanding is correct, calling ```RDD.first()``` doesn't materialize (read in) the full RDD, right? I saw a ```first``` being called

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50203116 QA results for PR 1338:br- This patch FAILED unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread JoshRosen
Github user JoshRosen commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50206155 If you call `first()` on an RDD, we only perform as much computation as necessary to return the first result. If the RDD's lineage chain contains a shuffle stage, we

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50207653 @JoshRosen hi, thanks for your detailed explanation. I'm not sure I get your point or maybe I didn't make myself clear. I'm mainly concerned about the case of reading

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50208305 @JoshRosen BTW, thanks for the detailed explanation on serialization! --- If your project is set up for it, you can reply to this email and have your reply appear on

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread JoshRosen
Github user JoshRosen commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50208947 @kanzhang I think it's unsafe to unpersist the read job's Hadoop configuration. Even if we cached the results of the read, I think we'd still need to keep a copy of

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50210598 Even if we cached the results of the read, I think we'd still need to keep a copy of the configuration in case any of those cached partitions are lost and we need to

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread JoshRosen
Github user JoshRosen commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50211345 Whoops, I was thinking of `destroy` instead of `unpersist.` Since the driver keeps a copy of the broadcast variable, it should always be safe to unpersist. --- If

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread kanzhang
Github user kanzhang commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15431198 --- Diff: python/pyspark/rdd.py --- @@ -964,6 +964,106 @@ def first(self): return self.take(1)[0] +def

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50219459 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17213/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50220618 QA results for PR 1338:br- This patch PASSES unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50223494 QA tests have started for PR 1338. This patch DID NOT merge cleanly! brView progress:

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-25 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50223574 Updated the patch to allow users to specify batch size when reading in a sequence file. When batch size is 1, the pickled data is unbatched (and custom Writables are

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread MLnick
Github user MLnick commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50068945 Ok I've been through it in detail and everything looks good, with the exception of one issue that I picked up. The saving currently inherently assumes that the

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50078131 @MLnick Thanks, Nick! Oops, I totally forgot about re-serialization, which I specifically dealt with when I worked on ```saveAsPickleFile```. Will update now. --- If

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50090301 Update addressing serialization issue raised by Nick, along with tests. --- If your project is set up for it, you can reply to this email and have your reply appear on

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50090439 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17141/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50093417 There is one additional issue though. Currently, when reading the Python rdd is not batch serialized. This will trigger re-serialization upon writing the same rdd,

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50093703 QA results for PR 1338:br- This patch PASSES unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15385187 --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala --- @@ -92,6 +104,46 @@ private[python] class DefaultConverter extends

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15386068 --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala --- @@ -92,6 +104,46 @@ private[python] class DefaultConverter extends

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread kanzhang
Github user kanzhang commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15386313 --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala --- @@ -92,6 +104,46 @@ private[python] class DefaultConverter extends

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50108287 Uploaded a new patch adding batch serialization to Python when reading sequence files. For some reason, the test on custom class (Java Bean) no longer works and I

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50108508 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17164/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50109757 More on the JavaBean test failure, it seems for RDD[(key, Bean)], all the keys in a partition (or batch, I didn't test partition size batch size) are paired with the

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-24 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-50110269 QA results for PR 1338:br- This patch FAILED unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-23 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49843828 The style check error is different, see https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16994/consoleFull. It's a bit hidden in there but it says:

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-23 Thread mateiz
Github user mateiz commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15275379 --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala --- @@ -31,13 +31,14 @@ import org.apache.spark.annotation.Experimental

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-23 Thread mateiz
Github user mateiz commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15275404 --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala --- @@ -31,13 +31,14 @@ import org.apache.spark.annotation.Experimental

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-23 Thread kanzhang
Github user kanzhang commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15296619 --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala --- @@ -31,13 +31,14 @@ import org.apache.spark.annotation.Experimental

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-23 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49895962 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17042/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-23 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49908850 QA results for PR 1338:br- This patch FAILED unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-23 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49912846 The latest test failure in ```org.apache.spark.streaming.CheckpointSuite``` seems unrelated. --- If your project is set up for it, you can reply to this email and have

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-22 Thread MLnick
Github user MLnick commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49708463 @kanzhang @mateiz Yeah this is one issue with Pyrolite vs MsgPack. MsgPack supported case classes out the box, which would likely be a bit more common that beans.

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-22 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49807264 @MLnick I merely removed it from programming guide. The functionality (and your test) is still there should anyone wants to try it. @mateiz I meant when

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-22 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49808256 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16994/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-22 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49808354 QA results for PR 1338:br- This patch FAILED unit tests.br- This patch merges cleanlybr- This patch adds no public classesbrbrFor more information see test

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-22 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49810291 Major changes for the updated patch. 1. Replaced doctests with standalone tests 2. Fixed converter for converting BytesWritables and added read/write tests

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-22 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49810818 @pwendell I renamed file HBaseConverter.scala to HBaseConverters.scala. Now I failed Scala style checks. How can I fix it? Thx. --- If your project is set up for it,

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-21 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49675615 Regarding the JavaBeans, is there a reason to believe Pyrolite won't support them in the future? Or are you just suggesting to remove it because we can't also save data?

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-20 Thread kanzhang
Github user kanzhang commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15150914 --- Diff: docs/programming-guide.md --- @@ -403,31 +403,30 @@ PySpark SequenceFile support loads an RDD within Java, and pickles the resulting

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-20 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49554126 @MLnick I'm thinking of removing the tests and programming guide entry for custom classes (JavaBeans). It seems to be a feature of Pyrolite and I can't think of any

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-20 Thread mateiz
Github user mateiz commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15152359 --- Diff: docs/programming-guide.md --- @@ -403,31 +403,30 @@ PySpark SequenceFile support loads an RDD within Java, and pickles the resulting

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-20 Thread kanzhang
Github user kanzhang commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r15153275 --- Diff: docs/programming-guide.md --- @@ -403,31 +403,30 @@ PySpark SequenceFile support loads an RDD within Java, and pickles the resulting

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-16 Thread MLnick
Github user MLnick commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-49198901 Great - I will review in more detail after that. Would be great to get this merged before 1.1 freeze so PySpark I/O for inputformat and outputformat is in for the

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-14 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48962322 @davies @mateiz @MLnick thanks for the review and suggestions. I'll try to add standalone tests for every data type. --- If your project is set up for it, you can

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-14 Thread kanzhang
Github user kanzhang commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48971710 @MLnick I'll see if I can add couple output converter examples as well. Thx. --- If your project is set up for it, you can reply to this email and have your reply

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-12 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48818290 Jenkins, retest this please --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-12 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48818358 QA tests have started for PR 1338. This patch merges cleanly. brView progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16588/consoleFull ---

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-12 Thread mateiz
Github user mateiz commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r14854296 --- Diff: docs/programming-guide.md --- @@ -403,31 +403,30 @@ PySpark SequenceFile support loads an RDD within Java, and pickles the resulting

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-12 Thread mateiz
Github user mateiz commented on a diff in the pull request: https://github.com/apache/spark/pull/1338#discussion_r14854311 --- Diff: python/pyspark/tests.py --- @@ -273,6 +274,14 @@ def test_sequencefiles(self): (3, {2.0: u'dd'})]

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-12 Thread mateiz
Github user mateiz commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48818519 This looks awesome, thanks for putting it together! One comment I have though is that we should add more test coverage, to make sure we cover all the data types

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-12 Thread MLnick
Github user MLnick commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48820160 I have had a quick look over and will try to do a more detailed one this weekend. High level looks good, 2 comments so far: 1. Agree with Matei that I

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-12 Thread SparkQA
Github user SparkQA commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48820892 QA results for PR 1338:br- This patch PASSES unit tests.br- This patch merges cleanlybr- This patch adds the following public classes (experimental):br tddict of

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-11 Thread davies
Github user davies commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48783095 LGTM, awesome! --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48501125 Merged build triggered. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48501142 Merged build started. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not

[GitHub] spark pull request: [SPARK-2024] Add saveAsSequenceFile to PySpark

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request: https://github.com/apache/spark/pull/1338#issuecomment-48508588 Merged build finished. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not

  1   2   >