aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py.Marcelo Vanzin2015-09-151-10/+0
| | | | | | | | | | | | | | | This change does two things: - tag a few tests and adds the mechanism in the build to be able to disable those tags, both in maven and sbt, for both junit and scalatest suites. - add some logic to run-tests.py to disable some tags depending on what files have changed; that's used to disable expensive tests when a module hasn't explicitly been changed, to speed up testing for changes that don't directly affect those modules. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #8437 from vanzin/test-tags.
* Update version to 1.6.0-SNAPSHOT.Reynold Xin2015-09-152-2/+2
| | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #8350 from rxin/1.6.
* [SPARK-9851] Support submitting map stages individually in DAGSchedulerMatei Zaharia2015-09-1412-63/+710
| | | | | | | | | | This patch adds support for submitting map stages in a DAG individually so that we can make downstream decisions after seeing statistics about their output, as part of SPARK-9850. I also added more comments to many of the key classes in DAGScheduler. By itself, the patch is not super useful except maybe to switch between a shuffle and broadcast join, but with the other subtasks of SPARK-9850 we'll be able to do more interesting decisions. The main entry point is SparkContext.submitMapStage, which lets you run a map stage and see stats about the map output sizes. Other stats could also be collected through accumulators. See AdaptiveSchedulingSuite for a short example. Author: Matei Zaharia <matei@databricks.com> Closes #8180 from mateiz/spark-9851.
* [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the ↵Andrew Or2015-09-141-8/+15
| | | | | | | | | | test (round 2) This is a follow-up patch to #8723. I missed one case there. Author: Andrew Or <andrew@databricks.com> Closes #8727 from andrewor14/fix-threading-suite.
* [SPARK-10543] [CORE] Peak Execution Memory Quantile should be Per-task BasisForest Fang2015-09-142-8/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | | | Read `PEAK_EXECUTION_MEMORY` using `update` to get per task partial value instead of cumulative value. I tested with this workload: ```scala val size = 1000 val repetitions = 10 val data = sc.parallelize(1 to size, 5).map(x => (util.Random.nextInt(size / repetitions),util.Random.nextDouble)).toDF("key", "value") val res = data.toDF.groupBy("key").agg(sum("value")).count ``` Before: ![image](https://cloud.githubusercontent.com/assets/4317392/9828197/07dd6874-58b8-11e5-9bd9-6ba927c38b26.png) After: ![image](https://cloud.githubusercontent.com/assets/4317392/9828151/a5ddff30-58b7-11e5-8d31-eda5dc4eae79.png) Tasks view: ![image](https://cloud.githubusercontent.com/assets/4317392/9828199/17dc2b84-58b8-11e5-92a8-be89ce4d29d1.png) cc andrewor14 I appreciate if you can give feedback on this since I think you introduced display of this metric. Author: Forest Fang <forest.fang@outlook.com> Closes #8726 from saurfang/stagepage.
* [SPARK-10576] [BUILD] Move .java files out of src/main/scalaSean Owen2015-09-144-0/+0
| | | | | | | | Move .java files in `src/main/scala` to `src/main/java` root, except for `package-info.java` (to stay next to package.scala) Author: Sean Owen <sowen@cloudera.com> Closes #8736 from srowen/SPARK-10576.
* [SPARK-9899] [SQL] log warning for direct output committer with speculation ↵Wenchen Fan2015-09-141-6/+38
| | | | | | | | | | | | | | | | enabled This is a follow-up of https://github.com/apache/spark/pull/8317. When speculation is enabled, there may be multiply tasks writing to the same path. Generally it's OK as we will write to a temporary directory first and only one task can commit the temporary directory to target path. However, when we use direct output committer, tasks will write data to target path directly without temporary directory. This causes problems like corrupted data. Please see [PR comment](https://github.com/apache/spark/pull/8191#issuecomment-131598385) for more details. Unfortunately, we don't have a simple flag to tell if a output committer will write to temporary directory or not, so for safety, we have to disable any customized output committer when `speculation` is true. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #8687 from cloud-fan/direct-committer.
* [SPARK-10330] Add Scalastyle rule to require use of SparkHadoopUtil ↵Josh Rosen2015-09-125-9/+17
| | | | | | | | | | JobContext methods This is a followup to #8499 which adds a Scalastyle rule to mandate the use of SparkHadoopUtil's JobContext accessor methods and fixes the existing violations. Author: Josh Rosen <joshrosen@databricks.com> Closes #8521 from JoshRosen/SPARK-10330-part2.
* [SPARK-10547] [TEST] Streamline / improve style of Java API testsSean Owen2015-09-121-227/+224
| | | | | | | | Fix a few Java API test style issues: unused generic types, exceptions, wrong assert argument order Author: Sean Owen <sowen@cloudera.com> Closes #8706 from srowen/SPARK-10547.
* [SPARK-10554] [CORE] Fix NPE with ShutdownHookNithin Asokan2015-09-121-1/+3
| | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-10554 Fixes NPE when ShutdownHook tries to cleanup temporary folders Author: Nithin Asokan <Nithin.Asokan@Cerner.com> Closes #8720 from nasokan/SPARK-10554.
* [SPARK-10566] [CORE] SnappyCompressionCodec init exception handling masks ↵Daniel Imfeld2015-09-121-1/+1
| | | | | | | | | | | | | | important error information When throwing an IllegalArgumentException in SnappyCompressionCodec.init, chain the existing exception. This allows potentially important debugging info to be passed to the user. Manual testing shows the exception chained properly, and the test suite still looks fine as well. This contribution is my original work and I license the work to the project under the project's open source license. Author: Daniel Imfeld <daniel@danielimfeld.com> Closes #8725 from dimfeld/dimfeld-patch-1.
* [SPARK-10564] ThreadingSuite: assertion failures in threads don't fail the testAndrew Or2015-09-111-23/+45
| | | | | | | | This commit ensures if an assertion fails within a thread, it will ultimately fail the test. Otherwise we end up potentially masking real bugs by not propagating assertion failures properly. Author: Andrew Or <andrew@databricks.com> Closes #8723 from andrewor14/fix-threading-suite.
* [SPARK-10546] Check partitionId's range in ExternalSorter#spill()tedyu2015-09-111-0/+2
| | | | | | | | | | | | | See this thread for background: http://search-hadoop.com/m/q3RTt0rWvIkHAE81 We should check the range of partition Id and provide meaningful message through exception. Alternatively, we can use abs() and modulo to force the partition Id into legitimate range. However, expectation is that user should correct the logic error in his / her code. Author: tedyu <yuzhihong@gmail.com> Closes #8703 from tedyu/master.
* [SPARK-9043] Serialize key, value and combiner classes in ShuffleDependencyMatt Massie2015-09-108-22/+167
| | | | | | | | | | | | | | | | | | | ShuffleManager implementations are currently not given type information for the key, value and combiner classes. Serialization of shuffle objects relies on objects being JavaSerializable, with methods defined for reading/writing the object or, alternatively, serialization via Kryo which uses reflection. Serialization systems like Avro, Thrift and Protobuf generate classes with zero argument constructors and explicit schema information (e.g. IndexedRecords in Avro have get, put and getSchema methods). By serializing the key, value and combiner class names in ShuffleDependency, shuffle implementations will have access to schema information when registerShuffle() is called. Author: Matt Massie <massie@cs.berkeley.edu> Closes #7403 from massie/shuffle-classtags.
* [SPARK-10049] [SPARKR] Support collecting data of ArraryType in DataFrame.Sun Rui2015-09-102-85/+145
| | | | | | | | | | | | | | this PR : 1. Enhance reflection in RBackend. Automatically matching a Java array to Scala Seq when finding methods. Util functions like seq(), listToSeq() in R side can be removed, as they will conflict with the Serde logic that transferrs a Scala seq to R side. 2. Enhance the SerDe to support transferring a Scala seq to R side. Data of ArrayType in DataFrame after collection is observed to be of Scala Seq type. 3. Support ArrayType in createDataFrame(). Author: Sun Rui <rui.sun@intel.com> Closes #8458 from sun-rui/SPARK-10049.
* [SPARK-10514] [MESOS] waiting for min no of total cores acquired by Spark by ↵Akash Mishra2015-09-101-0/+4
| | | | | | | | | | | | | | implementing the sufficientResourcesRegistered method spark.scheduler.minRegisteredResourcesRatio configuration parameter works for YARN mode but not for Mesos Coarse grained mode. If the parameter specified default value of 0 will be set for spark.scheduler.minRegisteredResourcesRatio in base class and this method will always return true. There are no existing test for YARN mode too. Hence not added test for the same. Author: Akash Mishra <akash.mishra20@gmail.com> Closes #8672 from SleepyThread/master.
* [SPARK-6350] [MESOS] Fine-grained mode scheduler respects mesosExecutor.coresIulian Dragos2015-09-102-3/+33
| | | | | | | | | | This is a regression introduced in #4960, this commit fixes it and adds a test. tnachen andrewor14 please review, this should be an easy one. Author: Iulian Dragos <jaguarul@gmail.com> Closes #8653 from dragos/issue/mesos/fine-grained-maxExecutorCores.
* [SPARK-8167] Make tasks that fail from YARN preemption not fail jobmcheah2015-09-1015-55/+186
| | | | | | | | | | | | | | | | | The architecture is that, in YARN mode, if the driver detects that an executor has disconnected, it asks the ApplicationMaster why the executor died. If the ApplicationMaster is aware that the executor died because of preemption, all tasks associated with that executor are not marked as failed. The executor is still removed from the driver's list of available executors, however. There's a few open questions: 1. Should standalone mode have a similar "get executor loss reason" as well? I localized this change as much as possible to affect only YARN, but there could be a valid case to differentiate executor losses in standalone mode as well. 2. I make a pretty strong assumption in YarnAllocator that getExecutorLossReason(executorId) will only be called once per executor id; I do this so that I can remove the metadata from the in-memory map to avoid object accumulation. It's not clear if I'm being overly zealous to save space, however. cc vanzin specifically for review because it collided with some earlier YARN scheduling work. cc JoshRosen because it's similar to output commit coordination we did in the past cc andrewor14 for our discussion on how to get executor exit codes and loss reasons Author: mcheah <mcheah@palantir.com> Closes #8007 from mccheah/feature/preemption-handling.
* [SPARK-10466] [SQL] UnsafeRow SerDe exception with data spillCheng Hao2015-09-101-0/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Data Spill with UnsafeRow causes assert failure. ``` java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:165) at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2.writeKey(UnsafeRowSerializer.scala:75) at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:180) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:688) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2$$anonfun$apply$1.apply(ExternalSorter.scala:687) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:687) at org.apache.spark.util.collection.ExternalSorter$$anonfun$writePartitionedFile$2.apply(ExternalSorter.scala:683) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at org.apache.spark.util.collection.ExternalSorter.writePartitionedFile(ExternalSorter.scala:683) at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:80) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:88) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) ``` To reproduce that with code (thanks andrewor14): ```scala bin/spark-shell --master local --conf spark.shuffle.memoryFraction=0.005 --conf spark.shuffle.sort.bypassMergeThreshold=0 sc.parallelize(1 to 2 * 1000 * 1000, 10) .map { i => (i, i) }.toDF("a", "b").groupBy("b").avg().count() ``` Author: Cheng Hao <hao.cheng@intel.com> Closes #8635 from chenghao-intel/unsafe_spill.
* [SPARK-9730] [SQL] Add Full Outer Join support for SortMergeJoinLiang-Chi Hsieh2015-09-091-0/+11
| | | | | | | | | | | | | | | This PR is based on #8383 , thanks to viirya JIRA: https://issues.apache.org/jira/browse/SPARK-9730 This patch adds the Full Outer Join support for SortMergeJoin. A new class SortMergeFullJoinScanner is added to scan rows from left and right iterators. FullOuterIterator is simply a wrapper of type RowIterator to consume joined rows from SortMergeFullJoinScanner. Closes #8383 Author: Liang-Chi Hsieh <viirya@appier.com> Author: Davies Liu <davies@databricks.com> Closes #8579 from davies/smj_fullouter.
* [SPARK-10227] fatal warnings with sbt on Scala 2.11Luc Bourlier2015-09-0932-86/+93
| | | | | | | | | | | The bulk of the changes are on `transient` annotation on class parameter. Often the compiler doesn't generate a field for this parameters, so the the transient annotation would be unnecessary. But if the class parameter are used in methods, then fields are created. So it is safer to keep the annotations. The remainder are some potential bugs, and deprecated syntax. Author: Luc Bourlier <luc.bourlier@typesafe.com> Closes #8433 from skyluc/issue/sbt-2.11.
* [SPARK-9767] Remove ConnectionManager.Reynold Xin2015-09-0718-3213/+15
| | | | | | | | We introduced the Netty network module for shuffle in Spark 1.2, and has turned it on by default for 3 releases. The old ConnectionManager is difficult to maintain. If we merge the patch now, by the time it is released, it would be 1 yr for which ConnectionManager is off by default. It's time to remove it. Author: Reynold Xin <rxin@databricks.com> Closes #8161 from rxin/SPARK-9767.
* [SPARK-10454] [SPARK CORE] wait for empty event queuerobbins2015-09-041-0/+1
| | | | | | Author: robbins <robbins@uk.ibm.com> Closes #8605 from robbinspg/DAGSchedulerSuite-fix.
* [SPARK-9669] [MESOS] Support PySpark on Mesos cluster mode.Timothy Chen2015-09-042-16/+39
| | | | | | | | | Support running pyspark with cluster mode on Mesos! This doesn't upload any scripts, so if running in a remote Mesos requires the user to specify the script from a available URI. Author: Timothy Chen <tnachen@gmail.com> Closes #8349 from tnachen/mesos_python.
* [SPARK-10003] Improve readability of DAGSchedulerAndrew Or2015-09-031-37/+9
| | | | | | | | | | | | | | | | | Note: this is not intended to be in Spark 1.5! This patch rewrites some code in the `DAGScheduler` to make it more readable. In particular - there were blocks of code that are unnecessary and removed for simplicity - there were abstractions that are unnecessary and made the code hard to navigate - other minor changes Author: Andrew Or <andrew@databricks.com> Closes #8217 from andrewor14/dag-scheduler-readability and squashes the following commits: 57abca3 [Andrew Or] Move comment back into if case 574fb1e [Andrew Or] Merge branch 'master' of github.com:apache/spark into dag-scheduler-readability 64a9ed2 [Andrew Or] Remove unnecessary code + minor code rewrites
* [SPARK-10421] [BUILD] Exclude curator artifacts from tachyon dependencies.Marcelo Vanzin2015-09-031-0/+8
| | | | | | | | | | | This avoids them being mistakenly pulled instead of the newer ones that Spark actually uses. Spark only depends on these artifacts transitively, so sometimes maven just decides to pick tachyon's version of the dependency for whatever reason. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #8577 from vanzin/SPARK-10421.
* [SPARK-10435] Spark submit should fail fast for Mesos cluster mode with RAndrew Or2015-09-031-0/+3
| | | | | | | | It's not supported yet so we should error with a clear message. Author: Andrew Or <andrew@databricks.com> Closes #8590 from andrewor14/mesos-cluster-r-guard.
* [SPARK-9591] [CORE] Job may fail for exception during getting remote blockjeanlyn2015-09-033-2/+80
| | | | | | | | | [SPARK-9591](https://issues.apache.org/jira/browse/SPARK-9591) When we getting the broadcast variable, we can fetch the block form several location,but now when connecting the lost blockmanager(idle for enough time removed by driver when using dynamic resource allocate and so on) will cause task fail,and the worse case will cause the job fail. Author: jeanlyn <jeanlyn92@gmail.com> Closes #7927 from jeanlyn/catch_exception.
* [SPARK-10430] [CORE] Added hashCode methods in AccumulableInfo and ↵Vinod K C2015-09-034-1/+26
| | | | | | | | RDDOperationScope Author: Vinod K C <vinod.kc@huawei.com> Closes #8581 from vinodkc/fix_RDDOperationScope_Hashcode.
* [SPARK-9672] [MESOS] Don’t include SPARK_ENV_LOADED when passing env varsPat Shields2015-09-032-4/+25
| | | | | | | | This contribution is my original work and I license the work to the project under the project's open source license. Author: Pat Shields <yeoldefortran@gmail.com> Closes #7979 from pashields/env-loading-on-driver.
* [SPARK-10431] [CORE] Fix intermittent test failure. Wait for event queue to ↵robbins2015-09-031-0/+4
| | | | | | | | be clear Author: robbins <robbins@uk.ibm.com> Closes #8582 from robbinspg/InputOutputMetricsSuite.
* [SPARK-8951] [SPARKR] support Unicode characters in collect()CHOIJAEHONG2015-09-031-5/+4
| | | | | | | | | Spark gives an error message and does not show the output when a field of the result DataFrame contains characters in CJK. I changed SerDe.scala in order that Spark support Unicode characters when writes a string to R. Author: CHOIJAEHONG <redrock07@naver.com> Closes #7494 from CHOIJAEHONG1/SPARK-8951.
* [SPARK-10379] preserve first page in UnsafeShuffleExternalSorterDavies Liu2015-09-023-3/+8
| | | | | | Author: Davies Liu <davies@databricks.com> Closes #8543 from davies/preserve_page.
* [SPARK-10247] [CORE] improve readability of a test case in DAGSchedulerSuiteImran Rashid2015-09-021-10/+47
| | | | | | | | This is pretty minor, just trying to improve the readability of `DAGSchedulerSuite`, I figure every bit helps. Before whenever I read this test, I never knew what "should work" and "should be ignored" really meant -- this adds some asserts & updates comments to make it more clear. Also some reformatting per a suggestion from markhamstra on https://github.com/apache/spark/pull/7699 Author: Imran Rashid <irashid@cloudera.com> Closes #8434 from squito/SPARK-10247.
* Removed code duplication in ShuffleBlockFetcherIteratorEvan Racah2015-09-021-8/+10
| | | | | | | | Added fetchUpToMaxBytes() to prevent having to update both code blocks when a change is made. Author: Evan Racah <ejracah@gmail.com> Closes #8514 from eracah/master.
* [SPARK-8707] RDD#toDebugString fails if any cached RDD has invalid partitionsnavis.ryu2015-09-022-2/+6
| | | | | | | | Added numPartitions(evaluate: Boolean) to RDD. With "evaluate=true" the method is same with "partitions.length". With "evaluate=false", it checks checked-out or already evaluated partitions in the RDD to get number of partition. If it's not those cases, returns -1. RDDInfo.partitionNum calls numPartition only when it's accessed. Author: navis.ryu <navis@apache.org> Closes #7127 from navis/SPARK-8707.
* [SPARK-5945] Spark should not retry a stage infinitely on a FetchFailedExceptionIlya Ganelin2015-09-023-5/+320
| | | | | | | | | | | | The ```Stage``` class now tracks whether there were a sufficient number of consecutive failures of that stage to trigger an abort. To avoid an infinite loop of stage retries, we abort the job completely after 4 consecutive stage failures for one stage. We still allow more than 4 consecutive stage failures if there is an intervening successful attempt for the stage, so that in very long-lived applications, where a stage may get reused many times, we don't abort the job after failures that have been recovered from successfully. I've added test cases to exercise the most obvious scenarios. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5636 from ilganeli/SPARK-5945.
* [SPARK-10004] [SHUFFLE] Perform auth checks when clients read shuffle data.Marcelo Vanzin2015-09-022-2/+3
| | | | | | | | | | | | | | | To correctly isolate applications, when requests to read shuffle data arrive at the shuffle service, proper authorization checks need to be performed. This change makes sure that only the application that created the shuffle data can read from it. Such checks are only enabled when "spark.authenticate" is enabled, otherwise there's no secure way to make sure that the client is really who it says it is. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #8218 from vanzin/SPARK-10004.
* [SPARK-7336] [HISTORYSERVER] Fix bug that applications status incorrect on ↵Chuan Shao2015-09-021-5/+22
| | | | | | | | JobHistory UI. Author: ArcherShao <shaochuan@huawei.com> Closes #5886 from ArcherShao/SPARK-7336.
* [SPARK-4223] [CORE] Support * in acls.zhuol2015-09-012-4/+63
| | | | | | | | | | | | | | | SPARK-4223. Currently we support setting view and modify acls but you have to specify a list of users. It would be nice to support * meaning all users have access. Manual tests to verify that: "*" works for any user in: a. Spark ui: view and kill stage. Done. b. Spark history server. Done. c. Yarn application killing. Done. Author: zhuol <zhuol@yahoo-inc.com> Closes #8398 from zhuoliu/4223.
* [SPARK-10341] [SQL] fix memory starving in unsafe SMJDavies Liu2015-08-313-6/+42
| | | | | | | | | | | | In SMJ, the first ExternalSorter could consume all the memory before spilling, then the second can not even acquire the first page. Before we have a better memory allocator, SMJ should call prepare() before call any compute() of it's children. cc rxin JoshRosen Author: Davies Liu <davies@databricks.com> Closes #8511 from davies/smj_memory.
* [SPARK-8730] Fixes - Deser objects containing a primitive class attributeEugenCepoi2015-08-312-5/+40
| | | | | | Author: EugenCepoi <cepoi.eugen@gmail.com> Closes #7122 from EugenCepoi/master.
* [SPARK-10184] [CORE] Optimization for bounds determination in RangePartitionerihainan2015-08-301-1/+1
| | | | | | | | | | JIRA Issue: https://issues.apache.org/jira/browse/SPARK-10184 Change `cumWeight > target` to `cumWeight >= target` in `RangePartitioner.determineBounds` method to make the output partitions more balanced. Author: ihainan <ihainan72@gmail.com> Closes #8397 from ihainan/opt_for_rangepartitioner.
* [SPARK-9284] [TESTS] Allow all tests to run without an assembly.Marcelo Vanzin2015-08-282-2/+152
| | | | | | | | | | | | | | | | | | | | | | | | | | This change aims at speeding up the dev cycle a little bit, by making sure that all tests behave the same w.r.t. where the code to be tested is loaded from. Namely, that means that tests don't rely on the assembly anymore, rather loading all needed classes from the build directories. The main change is to make sure all build directories (classes and test-classes) are added to the classpath of child processes when running tests. YarnClusterSuite required some custom code since the executors are run differently (i.e. not through the launcher library, like standalone and Mesos do). I also found a couple of tests that could leak a SparkContext on failure, and added code to handle those. With this patch, it's possible to run the following command from a clean source directory and have all tests pass: mvn -Pyarn -Phadoop-2.4 -Phive-thriftserver install Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #7629 from vanzin/SPARK-9284.
* [SPARK-10295] [CORE] Dynamic allocation in Mesos does not release when RDDs ↵Sean Owen2015-08-281-5/+0
| | | | | | | | | | | | are cached Remove obsolete warning about dynamic allocation not working with cached RDDs See discussion in https://issues.apache.org/jira/browse/SPARK-10295 Author: Sean Owen <sowen@cloudera.com> Closes #8489 from srowen/SPARK-10295.
* [SPARK-10251] [CORE] some common types are not registered for Kryo Serializat…Ram Sriharsha2015-08-262-1/+64
| | | | | | | | …ion by default Author: Ram Sriharsha <rsriharsha@hw11853.local> Closes #8465 from harsha2010/SPARK-10251.
* [SPARK-10048] [SPARKR] Support arbitrary nested Java array in serde.Sun Rui2015-08-252-35/+58
| | | | | | | | | | | This PR: 1. supports transferring arbitrary nested array from JVM to R side in SerDe; 2. based on 1, collect() implemenation is improved. Now it can support collecting data of complex types from a DataFrame. Author: Sun Rui <rui.sun@intel.com> Closes #8276 from sun-rui/SPARK-10048.
* [SPARK-9613] [CORE] Ban use of JavaConversions and migrate all existing uses ↵Sean Owen2015-08-2574-377/+342
| | | | | | | | | | | | to JavaConverters Replace `JavaConversions` implicits with `JavaConverters` Most occurrences I've seen so far are necessary conversions; a few have been avoidable. None are in critical code as far as I see, yet. Author: Sean Owen <sowen@cloudera.com> Closes #8033 from srowen/SPARK-9613.
* Fixed a typo in DAGScheduler.ehnalis2015-08-251-7/+20
| | | | | | Author: ehnalis <zoltan.zvara@gmail.com> Closes #8308 from ehnalis/master.
* [DOC] add missing parameters in SparkContext.scala for scala docZhang, Liye2015-08-251-1/+14
| | | | | | Author: Zhang, Liye <liye.zhang@intel.com> Closes #8412 from liyezhang556520/minorDoc.