- Apr 21, 2015
-
-
Marcelo Vanzin authored
The default java options make the call to GenerateMIMAIgnore take forever to run since it's gc'ing all the time. Improve things by setting the perm gen size / max heap size to larger values. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #5615 from vanzin/gen-mima-fix and squashes the following commits: f44e921 [Marcelo Vanzin] [minor] [build] Set java options when generating mima ignores.
-
Josh Rosen authored
This patch modifies several shuffle-related code paths to share and re-use SerializerInstances instead of creating new ones. Some serializers, such as KryoSerializer or SqlSerializer, can be fairly expensive to create or may consume moderate amounts of memory, so it's probably best to avoid unnecessary serializer creation in hot code paths. The key change in this patch is modifying `getDiskWriter()` / `DiskBlockObjectWriter` to accept `SerializerInstance`s instead of `Serializer`s (which are factories for instances). This allows the disk writer's creator to decide whether the serializer instance can be shared or re-used. The rest of the patch modifies several write and read paths to use shared serializers. One big win is in `ShuffleBlockFetcherIterator`, where we used to create a new serializer per received block. Similarly, the shuffle write path used to create a new serializer per file even though in many cases only a single thread would be writing to a file at a time. I made a small serializer reuse optimization in CoarseGrainedExecutorBackend as well, since it seemed like a small and obvious improvement. Author: Josh Rosen <joshrosen@databricks.com> Closes #5606 from JoshRosen/SPARK-3386 and squashes the following commits: f661ce7 [Josh Rosen] Remove thread local; add comment instead 64f8398 [Josh Rosen] Use ThreadLocal for serializer instance in CoarseGrainedExecutorBackend aeb680e [Josh Rosen] [SPARK-3386] Reuse SerializerInstance in shuffle code paths
-
Cheng Hao authored
It's a bug while do query like: ```sql select d from (select explode(array(1,1)) d from src limit 1) t ``` And it will throws exception like: ``` org.apache.spark.sql.AnalysisException: cannot resolve 'd' given input columns _c0; line 1 pos 7 at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:48) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:45) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:50) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:249) at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$transformExpressionUp$1(QueryPlan.scala:103) at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2$$anonfun$apply$2.apply(QueryPlan.scala:117) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2.apply(QueryPlan.scala:116) at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) ``` To solve the bug, it requires code refactoring for UDTF The major changes are about: * Simplifying the UDTF development, UDTF will manage the output attribute names any more, instead, the `logical.Generate` will handle that properly. * UDTF will be asked for the output schema (data types) during the logical plan analyzing. Author: Cheng Hao <hao.cheng@intel.com> Closes #4602 from chenghao-intel/explode_bug and squashes the following commits: c2a5132 [Cheng Hao] add back resolved for Alias 556e982 [Cheng Hao] revert the unncessary change 002c361 [Cheng Hao] change the rule of resolved for Generate 04ae500 [Cheng Hao] add qualifier only for generator output 5ee5d2c [Cheng Hao] prepend the new qualifier d2e8b43 [Cheng Hao] Update the code as feedback ca5e7f4 [Cheng Hao] shrink the commits
-
Punya Biswal authored
liancheng mengxr this is similar to #5146. Author: Punya Biswal <pbiswal@palantir.com> Closes #5578 from punya/feature/SPARK-6996 and squashes the following commits: d56c3e0 [Punya Biswal] Fix imports c7e308b [Punya Biswal] Support java iterable types in POJOs 5e00685 [Punya Biswal] Support map types in java beans
-
Yin Huai authored
https://issues.apache.org/jira/browse/SPARK-6969 Author: Yin Huai <yhuai@databricks.com> Closes #5583 from yhuai/refreshTableRefreshDataCache and squashes the following commits: 1e5142b [Yin Huai] Add todo. 92b2498 [Yin Huai] Minor updates. 367df92 [Yin Huai] Recache data in the command of REFRESH TABLE.
-
Wenchen Fan authored
[SQL][minor] make it more clear that we only need to re-throw GetField exception for UnresolvedAttribute For `GetField` outside `UnresolvedAttribute`, we will throw exception in `Analyzer`. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #5588 from cloud-fan/tmp and squashes the following commits: 7ac74d2 [Wenchen Fan] small refactor
-
vidmantas zemleris authored
It looked weird that up to now there was no way in Spark's Scala API to access fields of `DataFrame/sql.Row` by name, only by their index. This tries to solve this issue. Author: vidmantas zemleris <vidmantas@vinted.com> Closes #5573 from vidma/features/row-with-named-fields and squashes the following commits: 6145ae3 [vidmantas zemleris] [SPARK-6994][SQL] Allow to fetch field values by name on Row 9564ebb [vidmantas zemleris] [SPARK-6994][SQL] Add fieldIndex to schema (StructType)
-
Prashant Sharma authored
[SPARK-7011] Build(compilation) fails with scala 2.11 option, because a protected[sql] type is accessed in ml package. [This](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala#L58) is where it is used and fails compilations at. Author: Prashant Sharma <prashant.s@imaginea.com> Closes #5593 from ScrapCodes/SPARK-7011/build-fix and squashes the following commits: e6d57a3 [Prashant Sharma] [SPARK-7011] Build fails with scala 2.11 option, because a protected[sql] type is accessed in ml package.
-
MechCoder authored
Since sparse matrices now support a isTransposed flag for row major data, DenseMatrices should do the same. Author: MechCoder <manojkumarsivaraj334@gmail.com> Closes #5455 from MechCoder/spark-6845 and squashes the following commits: 525c370 [MechCoder] minor 004a37f [MechCoder] Cast boolean to int 151f3b6 [MechCoder] [WIP] Add isTransposed to pickle DenseMatrix cc0b90a [MechCoder] [SPARK-6845] Add isTranposed flag to DenseMatrix
-
emres authored
SPARK-3276 Added a new configuration parameter ``spark.streaming.minRememberDuration``, with a default value of 1 minute. So that when a Spark Streaming application is started, an arbitrary number of minutes can be taken as threshold for remembering. Author: emres <emre.sevinc@gmail.com> Closes #5438 from emres/SPARK-3276 and squashes the following commits: 766f938 [emres] SPARK-3276 Switched to using newly added getTimeAsSeconds method. affee1d [emres] SPARK-3276 Changed the property name and variable name for minRememberDuration c9d58ca [emres] SPARK-3276 Minor code re-formatting. 1c53ba9 [emres] SPARK-3276 Started to use ssc.conf rather than ssc.sparkContext.getConf, and also getLong method directly. bfe0acb [emres] SPARK-3276 Moved the minRememberDurationMin to the class daccc82 [emres] SPARK-3276 Changed the property name to reflect the unit of value and reduced number of fields. 43cc1ce [emres] SPARK-3276 Added a new configuration parameter spark.streaming.minRemember duration, with a default value of 1 minute.
-
Kay Ousterhout authored
CoGroupPartition, part of CoGroupedRDD, includes references to each RDD that the CoGroupedRDD narrowly depends on, and a reference to the ShuffleHandle. The partition is serialized separately from the RDD, so when the RDD and partition arrive on the worker, the references in the partition and in the RDD no longer point to the same object. This is a relatively minor performance issue (the closure can be 2x larger than it needs to be because the rdds and partitions are serialized twice; see numbers below) but is more annoying as a developer issue (this is where I ran into): if any state is stored in the RDD or ShuffleHandle on the worker side, subtle bugs can appear due to the fact that the references to the RDD / ShuffleHandle in the RDD and in the partition point to separate objects. I'm not sure if this is enough of a potential future problem to fix this old and central part of the code, so hoping to get input from others here. I did some simple experiments to see how much this effects closure size. For this example: $ val a = sc.parallelize(1 to 10).map((_, 1)) $ val b = sc.parallelize(1 to 2).map(x => (x, 2*x)) $ a.cogroup(b).collect() the closure was 1902 bytes with current Spark, and 1129 bytes after my change. The difference comes from eliminating duplicate serialization of the shuffle handle. For this example: $ val sortedA = a.sortByKey() $ val sortedB = b.sortByKey() $ sortedA.cogroup(sortedB).collect() the closure was 3491 bytes with current Spark, and 1333 bytes after my change. Here, the difference comes from eliminating duplicate serialization of the two RDDs for the narrow dependencies. The ShuffleHandle includes the ShuffleDependency, so this difference will get larger if a ShuffleDependency includes a serializer, a key ordering, or an aggregator (all set to None by default). It would also get bigger for a big RDD -- although I can't think of any examples where the RDD object gets large. The difference is not affected by the size of the function the user specifies, which (based on my understanding) is typically the source of large task closures. Author: Kay Ousterhout <kayousterhout@gmail.com> Closes #4145 from kayousterhout/SPARK-5360 and squashes the following commits: 85156c3 [Kay Ousterhout] Better comment the narrowDeps parameter cff0209 [Kay Ousterhout] Fixed spelling issue 658e1af [Kay Ousterhout] [SPARK-5360] Eliminate duplicate objects in serialized CoGroupedRDD
-
David McGuire authored
A simple truncation in integer division (on rates over 1000 messages / second) causes the existing implementation to sleep for 0 milliseconds, then call itself recursively; this causes what is essentially an infinite recursion, since the base case of the calculated amount of time having elapsed can't be reached before available stack space is exhausted. A fix to this truncation error is included in this patch. However, even with the defect patched, the accuracy of the existing implementation is abysmal (the error bounds of the original test were effectively [-30%, +10%], although this fact was obscured by hard-coded error margins); as such, when the error bounds were tightened down to [-5%, +5%], the existing implementation failed to meet the new, tightened, requirements. Therefore, an industry-vetted solution (from Guava) was used to get the adapted tests to pass. Author: David McGuire <david.mcguire2@nike.com> Closes #5559 from dmcguire81/master and squashes the following commits: d29d2e0 [David McGuire] Back out to +/-5% error margins, for flexibility in timing 8be6934 [David McGuire] Fix spacing per code review 90e98b9 [David McGuire] Address scalastyle errors 29011bd [David McGuire] Further ratchet down the error margins b33b796 [David McGuire] Eliminate dependency on even distribution by BlockGenerator 8f2934b [David McGuire] Remove arbitrary thread timing / cooperation code 70ee310 [David McGuire] Use Thread.yield(), since Thread.sleep(0) is system-dependent 82ee46d [David McGuire] Replace guard clause with nested conditional 2794717 [David McGuire] Replace the RateLimiter with the Guava implementation 38f3ca8 [David McGuire] Ratchet down the error rate to +/- 5%; tests fail 24b1bc0 [David McGuire] Fix truncation in integer division causing infinite recursion d6e1079 [David McGuire] Stack overflow error in RateLimiter on rates over 1000/s
-
Yanbo Liang authored
Model import/export for IsotonicRegression Author: Yanbo Liang <ybliang8@gmail.com> Closes #5270 from yanboliang/spark-5990 and squashes the following commits: 872028d [Yanbo Liang] fix code style f80ec1b [Yanbo Liang] address comments 49600cc [Yanbo Liang] address comments 429ff7d [Yanbo Liang] store each interval as a record 2b2f5a1 [Yanbo Liang] Model import/export for IsotonicRegression
-
Davies Liu authored
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
-
zsxwing authored
Deprecated `spark.akka.num.retries`, `spark.akka.retry.wait`, `spark.akka.askTimeout`, `spark.akka.lookupTimeout`, and added `spark.rpc.num.retries`, `spark.rpc.retry.wait`, `spark.rpc.askTimeout`, `spark.rpc.lookupTimeout`. Author: zsxwing <zsxwing@gmail.com> Closes #5595 from zsxwing/SPARK-6490 and squashes the following commits: e0d80a9 [zsxwing] Use getTimeAsMs and getTimeAsSeconds and other minor fixes 31dbe69 [zsxwing] Add spark.rpc.* and deprecate spark.akka.*
-
- Apr 20, 2015
-
-
Liang-Chi Hsieh authored
JIRA https://issues.apache.org/jira/browse/SPARK-6635 Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #5541 from viirya/replace_with_column and squashes the following commits: b539c7b [Liang-Chi Hsieh] For comment. 72f35b1 [Liang-Chi Hsieh] DataFrame.withColumn can replace original column with identical column name.
-
Yin Huai authored
JIRA: https://issues.apache.org/jira/browse/SPARK-6368 Author: Yin Huai <yhuai@databricks.com> Closes #5497 from yhuai/serializer2 and squashes the following commits: da562c5 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2 50e0c3d [Yin Huai] When no filed is emitted to shuffle, use SparkSqlSerializer for now. 9f1ed92 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2 6d07678 [Yin Huai] Address comments. 4273b8c [Yin Huai] Enabled SparkSqlSerializer2. 09e587a [Yin Huai] Remove TODO. 791b96a [Yin Huai] Use UTF8String. 60a1487 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2 3e09655 [Yin Huai] Use getAs for Date column. 43b9fb4 [Yin Huai] Test. 8297732 [Yin Huai] Fix test. c9373c8 [Yin Huai] Support DecimalType. 2379eeb [Yin Huai] ASF header. 39704ab [Yin Huai] Specialized serializer for Exchange.
-
BenFradet authored
The commit message is pretty self-explanatory. Author: BenFradet <benjamin.fradet@gmail.com> Closes #5600 from BenFradet/master and squashes the following commits: 108492d [BenFradet] [doc][streaming] Fixed broken link in mllib section
-
Eric Chiang authored
The contribution is my original work. I license the work to the project under the project's open source license. Small typo in the programming guide. Author: Eric Chiang <eric.chiang.m@gmail.com> Closes #5599 from ericchiang/docs-typo and squashes the following commits: 1177942 [Eric Chiang] fixed doc
-
Liang-Chi Hsieh authored
It should load from `testInput` instead of `input` for test data. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #5594 from viirya/use_testinput and squashes the following commits: 5e8b174 [Liang-Chi Hsieh] Fix style. b60b475 [Liang-Chi Hsieh] Use testInput.
-
Elisey Zanko authored
Author: Elisey Zanko <elisey.zanko@gmail.com> Closes #5361 from 31z4/spark-6661 and squashes the following commits: 73c5d79 [Elisey Zanko] Python type errors should print type, not object
-
Aaron Davidson authored
[SPARK-7003] Improve reliability of connection failure detection between Netty block transfer service endpoints Currently we rely on the assumption that an exception will be raised and the channel closed if two endpoints cannot communicate over a Netty TCP channel. However, this guarantee does not hold in all network environments, and [SPARK-6962](https://issues.apache.org/jira/browse/SPARK-6962) seems to point to a case where only the server side of the connection detected a fault. This patch improves robustness of fetch/rpc requests by having an explicit timeout in the transport layer which closes the connection if there is a period of inactivity while there are outstanding requests. NB: This patch is actually only around 50 lines added if you exclude the testing-related code. Author: Aaron Davidson <aaron@databricks.com> Closes #5584 from aarondav/timeout and squashes the following commits: 8699680 [Aaron Davidson] Address Reynold's comments 37ce656 [Aaron Davidson] [SPARK-7003] Improve reliability of connection failure detection between Netty block transfer service endpoints
-
jrabary authored
The current implementation call the default constructor of mllib.feature.StandarScaler without the possibility to specify withMean or withStd options. Author: jrabary <Jaonary@gmail.com> Closes #4704 from jrabary/master and squashes the following commits: fae8568 [jrabary] style fix 8896b0e [jrabary] Comments fix ef96d73 [jrabary] style fix 8e52607 [jrabary] style fix edd9d48 [jrabary] Fix default param initialization 17e1a76 [jrabary] Fix default param initialization 298f405 [jrabary] Typo fix 45ed914 [jrabary] Add withMean and withStd params to StandarScaler
-
- Apr 19, 2015
-
-
dobashim authored
* Fix the page title in Isotonic regression documents (Naive Bayes -> Isotonic regression) * Add a newline character at the end of the file Author: dobashim <dobashim@oss.nttdata.co.jp> Closes #5581 from dobashim/master and squashes the following commits: d54a041 [dobashim] Fix typo of the page title in Isotonic regression documents
-
zsxwing authored
Title says it all. cc rxin tdas Author: zsxwing <zsxwing@gmail.com> Closes #5554 from zsxwing/SPARK-6979 and squashes the following commits: 5304350 [zsxwing] Fix NotSerializableException e9d3479 [zsxwing] Add blank lines 633e279 [zsxwing] Fix NotSerializableException e496ace [zsxwing] Replace JobGenerator.eventActor with EventLoop ec6ec58 [zsxwing] Fix the import order ce0fa73 [zsxwing] Replace JobScheduler.eventActor with EventLoop
-
zsxwing authored
A subtask of [SPARK-5293](https://issues.apache.org/jira/browse/SPARK-5293) Author: zsxwing <zsxwing@gmail.com> Closes #5557 from zsxwing/SPARK-6983 and squashes the following commits: e777e9f [zsxwing] Update ReceiverTrackerActor to use the new Rpc interface
-
zsxwing authored
If `StreamingKMeans` is not `Serializable`, we cannot do checkpoint for applications that using `StreamingKMeans`. So we should make it `Serializable`. Author: zsxwing <zsxwing@gmail.com> Closes #5582 from zsxwing/SPARK-6998 and squashes the following commits: 67c2a14 [zsxwing] Make StreamingKMeans 'Serializable'
-
GuoQiang Li authored
cc andrewor14 Author: GuoQiang Li <witgo@qq.com> Closes #5548 from witgo/SPARK-6963 and squashes the following commits: 964aea7 [GuoQiang Li] review commits b08b3c9 [GuoQiang Li] Flaky test: o.a.s.ContextCleanerSuite automatically cleanup checkpoint
-
- Apr 18, 2015
-
-
Olivier Girardot authored
The default method will use Guava's Ordering instead of java.util.Comparator.naturalOrder() because it's not available in Java 7, only in Java 8. Author: Olivier Girardot <o.girardot@lateral-thoughts.com> Closes #5571 from ogirardot/master and squashes the following commits: 7fe2e9e [Olivier Girardot] SPARK-6993 : Add default min, max methods for JavaDoubleRDD
-
Gaurav Nanda authored
Just fixed a doc. Author: Gaurav Nanda <gaurav324@gmail.com> Closes #5576 from gaurav324/master and squashes the following commits: 8a7323f [Gaurav Nanda] Fixed doc
-
Nicholas Chammas authored
Per the discussion in the comments on [this commit](https://github.com/apache/spark/commit/f17d43b033d928dbc46aef8e367aa08902e698ad#commitcomment-10780649), this PR allows the Python lint script to reuse `pep8.py` when possible. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #5561 from nchammas/save-dem-pep8-bytes and squashes the following commits: b7c91e6 [Nicholas Chammas] reuse pep8.py
-
Marcelo Vanzin authored
My previous fix (force a selector wakeup) didn't seem to work since I ran into the hang again. So change the code a bit to be more explicit about the condition when the selector thread should exit. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #5566 from vanzin/conn-mgr-hang and squashes the following commits: ddb2c03 [Marcelo Vanzin] [core] [minor] Make sure ConnectionManager stops.
-
Olivier Girardot authored
This patch is fixing the Java examples for Spark SQL when defining programmatically a Schema and mapping Rows. Author: Olivier Girardot <o.girardot@lateral-thoughts.com> Closes #5569 from ogirardot/branch-1.3 and squashes the following commits: c29e58d [Olivier Girardot] SPARK-6992 : Fix documentation example for Spark SQL on StructType (cherry picked from commit c9b1ba4b16a7afe93d45bf75b128cc0dd287ded0) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
- Apr 17, 2015
-
-
jerryshao authored
`numExecutors` checking is failed when dynamic allocation is enabled with default configuration. Details can be seen is [SPARK-6975](https://issues.apache.org/jira/browse/SPARK-6975). sryza, please help me to review this, not sure is this the correct way, I think previous you change this part :) Author: jerryshao <saisai.shao@intel.com> Closes #5551 from jerryshao/SPARK-6975 and squashes the following commits: 4335da1 [jerryshao] Change according to the comments 77bdcbd [jerryshao] Fix argument validation error
-
Marcelo Vanzin authored
I didn't find many deprecated configs after a grep-based search, but the ones I could find were moved to the centralized location in SparkConf. While there, I deprecated a couple more HS configs that mentioned time units. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #5562 from vanzin/SPARK-5933 and squashes the following commits: dcb617e7 [Marcelo Vanzin] [SPARK-5933] [core] Move config deprecation warnings to SparkConf.
-
Jongyoul Lee authored
- Defined executorCores from "spark.mesos.executor.cores" - Changed the amount of mesosExecutor's cores to executorCores. - Added new configuration option on running-on-mesos.md Author: Jongyoul Lee <jongyoul@gmail.com> Closes #5063 from jongyoul/SPARK-6350 and squashes the following commits: 9238d6e [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs - Changed configuration name - Made mesosExecutorCores private 2d41241 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 89edb4f [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 8ba7694 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 7549314 [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Fixed docs 4ae7b0c [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Removed TODO c27efce [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Fixed Mesos*Suite for supporting integer WorkerOffers - Fixed Documentation 1fe4c03 [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Change available resources of cpus to integer value beacuse WorkerOffer support the amount cpus as integer value 5f3767e [Jongyoul Lee] Revert "[SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode" 4b7c69e [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Changed configruation name and description from "spark.mesos.executor.cores" to "spark.executor.frameworkCores" 0556792 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Defined executorCores from "spark.mesos.executor.cores" - Changed the amount of mesosExecutor's cores to executorCores. - Added new configuration option on running-on-mesos.md
-
Ilya Ganelin authored
I've added a static getOrCreate method to the static SparkContext object that allows one to either retrieve a previously created SparkContext or to instantiate a new one with the provided config. The method accepts an optional SparkConf to make usage intuitive. Still working on a test for this, basically want to create a new context from scratch, then ensure that subsequent calls don't overwrite that. Author: Ilya Ganelin <ilya.ganelin@capitalone.com> Closes #5501 from ilganeli/SPARK-6703 and squashes the following commits: db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
-
Reynold Xin authored
As a follow up PR to #5544. cc davies Author: Reynold Xin <rxin@databricks.com> Closes #5558 from rxin/sort-doc-improvement and squashes the following commits: f4c276f [Reynold Xin] Review feedback. d2dcf24 [Reynold Xin] Minor fix to SPARK-6958: Improve Python docstring for DataFrame.sort.
-
Olivier Girardot authored
This patch includes : * adding how to use map after an sql query using javaRDD * fixing the first few java examples that were written in Scala Thank you for your time, Olivier. Author: Olivier Girardot <o.girardot@lateral-thoughts.com> Closes #5564 from ogirardot/branch-1.3 and squashes the following commits: 9f8d60e [Olivier Girardot] SPARK-6988 : Fix documentation regarding DataFrames using the Java API (cherry picked from commit 6b528dc139da594ef2e651d84bd91fe0f738a39d) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
cafreeman authored
This PR pulls in recent changes in SparkR-pkg, including cartesian, intersection, sampleByKey, subtract, subtractByKey, except, and some API for StructType and StructField. Author: cafreeman <cfreeman@alteryx.com> Author: Davies Liu <davies@databricks.com> Author: Zongheng Yang <zongheng.y@gmail.com> Author: Shivaram Venkataraman <shivaram.venkataraman@gmail.com> Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Author: Sun Rui <rui.sun@intel.com> Closes #5436 from davies/R3 and squashes the following commits: c2b09be [Davies Liu] SQLTypes -> schema a5a02f2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R3 168b7fe [Davies Liu] sort generics b1fe460 [Davies Liu] fix conflict in README.md e74c04e [Davies Liu] fix schema.R 4f5ac09 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R5 41f8184 [Davies Liu] rm man ae78312 [Davies Liu] Merge pull request #237 from sun-rui/SPARKR-154_3 1bdcb63 [Zongheng Yang] Updates to README.md. 5a553e7 [cafreeman] Use object attribute instead of argument 71372d9 [cafreeman] Update docs and examples 8526d2e71 [cafreeman] Remove `tojson` functions 6ef5f2d [cafreeman] Fix spacing 7741d66 [cafreeman] Rename the SQL DataType function 141efd8 [Shivaram Venkataraman] Merge pull request #245 from hqzizania/upstream 9387402 [Davies Liu] fix style 40199eb [Shivaram Venkataraman] Move except into sorted position 07d0dbc [Sun Rui] [SPARKR-244] Fix test failure after integration of subtract() and subtractByKey() for RDD. 7e8caa3 [Shivaram Venkataraman] Merge pull request #246 from hlin09/fixCombineByKey ed66c81 [cafreeman] Update `subtract` to work with `generics.R` f3ba785 [cafreeman] Fixed duplicate export 275deb4 [cafreeman] Update `NAMESPACE` and tests 1a3b63d [cafreeman] new version of `CreateDF` 836c4bf [cafreeman] Update `createDataFrame` and `toDF` be5d5c1 [cafreeman] refactor schema functions 40338a4 [Zongheng Yang] Merge pull request #244 from sun-rui/SPARKR-154_5 20b97a6 [Zongheng Yang] Merge pull request #234 from hqzizania/assist ba54e34 [Shivaram Venkataraman] Merge pull request #238 from sun-rui/SPARKR-154_4 c9497a3 [Shivaram Venkataraman] Merge pull request #208 from lythesia/master b317aa7 [Zongheng Yang] Merge pull request #243 from hqzizania/master 136a07e [Zongheng Yang] Merge pull request #242 from hqzizania/stats cd66603 [cafreeman] new line at EOF 8b76e81 [Shivaram Venkataraman] Merge pull request #233 from redbaron/fail-early-on-missing-dep 7dd81b7 [cafreeman] Documentation 0e2a94f [cafreeman] Define functions for schema and fields
-