aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-9245] [MLLIB] LDA topic assignmentsJoseph K. Bradley2015-08-204-7/+74
| | | | | | | | | | For each (document, term) pair, return top topic. Note that instances of (doc, term) pairs within a document (a.k.a. "tokens") are exchangeable, so we should provide an estimate per document-term, rather than per token. CC: rotationsymmetry mengxr Author: Joseph K. Bradley <joseph@databricks.com> Closes #8329 from jkbradley/lda-topic-assignments.
* [SPARK-10108] Add since tags to mllib.featureMechCoder2015-08-209-11/+76
| | | | | | Author: MechCoder <manojkumarsivaraj334@gmail.com> Closes #8309 from MechCoder/tags_feature.
* [SPARK-10138] [ML] move setters to MultilayerPerceptronClassifier and add ↵Xiangrui Meng2015-08-202-27/+101
| | | | | | | | | | Java test suite Otherwise, setters do not return self type. jkbradley avulanov Author: Xiangrui Meng <meng@databricks.com> Closes #8342 from mengxr/SPARK-10138.
* [SQL] [MINOR] remove unnecessary classWenchen Fan2015-08-201-64/+0
| | | | | | | | This class is identical to `org.apache.spark.sql.execution.datasources.jdbc. DefaultSource` and is not needed. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #8334 from cloud-fan/minor.
* [SPARK-10126] [PROJECT INFRA] Fix typo in release-build.sh which broke ↵Josh Rosen2015-08-201-2/+2
| | | | | | | | | | snapshot publishing for Scala 2.11 The current `release-build.sh` has a typo which breaks snapshot publication for Scala 2.11. We should change the Scala version to 2.11 and clean before building a 2.11 snapshot. Author: Josh Rosen <joshrosen@databricks.com> Closes #8325 from JoshRosen/fix-2.11-snapshots.
* [SPARK-10136] [SQL] Fixes Parquet support for Avro array of primitive arrayCheng Lian2015-08-2013-844/+1718
| | | | | | | | I caught SPARK-10136 while adding more test cases to `ParquetAvroCompatibilitySuite`. Actual bug fix code lies in `CatalystRowConverter.scala`. Author: Cheng Lian <lian@databricks.com> Closes #8341 from liancheng/spark-10136/parquet-avro-nested-primitive-array.
* [SPARK-9982] [SPARKR] SparkR DataFrame fail to return data of Decimal typeAlex Shkurenko2015-08-201-0/+5
| | | | | | Author: Alex Shkurenko <ashkurenko@enova.com> Closes #8239 from ashkurenko/master.
* [MINOR] [SQL] Fix sphinx warnings in PySpark SQLMechCoder2015-08-202-5/+7
| | | | | | Author: MechCoder <manojkumarsivaraj334@gmail.com> Closes #8171 from MechCoder/sql_sphinx.
* [SPARK-10100] [SQL] Eliminate hash table lookup if there is no grouping key ↵Reynold Xin2015-08-202-10/+22
| | | | | | | | | | in aggregation. This improves performance by ~ 20 - 30% in one of my local test and should fix the performance regression from 1.4 to 1.5 on ss_max. Author: Reynold Xin <rxin@databricks.com> Closes #8332 from rxin/SPARK-10100.
* [SPARK-10092] [SQL] Multi-DB support follow up.Yin Huai2015-08-2016-94/+398
| | | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-10092 This pr is a follow-up one for Multi-DB support. It has the following changes: * `HiveContext.refreshTable` now accepts `dbName.tableName`. * `HiveContext.analyze` now accepts `dbName.tableName`. * `CreateTableUsing`, `CreateTableUsingAsSelect`, `CreateTempTableUsing`, `CreateTempTableUsingAsSelect`, `CreateMetastoreDataSource`, and `CreateMetastoreDataSourceAsSelect` all take `TableIdentifier` instead of the string representation of table name. * When you call `saveAsTable` with a specified database, the data will be saved to the correct location. * Explicitly do not allow users to create a temporary with a specified database name (users cannot do it before). * When we save table to metastore, we also check if db name and table name can be accepted by hive (using `MetaStoreUtils.validateName`). Author: Yin Huai <yhuai@databricks.com> Closes #8324 from yhuai/saveAsTableDB.
* [SPARK-10128] [STREAMING] Used correct classloader to deserialize WAL dataTathagata Das2015-08-191-2/+3
| | | | | | | | | | Recovering Kinesis sequence numbers from WAL leads to classnotfoundexception because the ObjectInputStream does not use the correct classloader and the SequenceNumberRanges class (in streaming-kinesis-asl package) cannot be found (added through spark-submit) while deserializing. The solution is to use `Thread.currentThread().getContextClassLoader` while deserializing. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8328 from tdas/SPARK-10128 and squashes the following commits: f19b1c2 [Tathagata Das] Used correct classloader to deserialize WAL data
* [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode.Timothy Chen2015-08-191-8/+11
| | | | | | | | | | Currently the spark applications can be queued to the Mesos cluster dispatcher, but when multiple jobs are in queue we don't handle removing jobs from the buffer correctly while iterating and causes null pointer exception. This patch copies the buffer before iterating them, so exceptions aren't thrown when the jobs are removed. Author: Timothy Chen <tnachen@gmail.com> Closes #8322 from tnachen/fix_cluster_mode.
* [SPARK-10125] [STREAMING] Fix a potential deadlock in JobGenerator.stopzsxwing2015-08-191-0/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Because `lazy val` uses `this` lock, if JobGenerator.stop and JobGenerator.doCheckpoint (JobGenerator.shouldCheckpoint has not yet been initialized) run at the same time, it may hang. Here are the stack traces for the deadlock: ```Java "pool-1-thread-1-ScalaTest-running-StreamingListenerSuite" #11 prio=5 os_prio=31 tid=0x00007fd35d094800 nid=0x5703 in Object.wait() [0x000000012ecaf000] java.lang.Thread.State: WAITING (on object monitor) at java.lang.Object.wait(Native Method) at java.lang.Thread.join(Thread.java:1245) - locked <0x00000007b5d8d7f8> (a org.apache.spark.util.EventLoop$$anon$1) at java.lang.Thread.join(Thread.java:1319) at org.apache.spark.util.EventLoop.stop(EventLoop.scala:81) at org.apache.spark.streaming.scheduler.JobGenerator.stop(JobGenerator.scala:155) - locked <0x00000007b5d8cea0> (a org.apache.spark.streaming.scheduler.JobGenerator) at org.apache.spark.streaming.scheduler.JobScheduler.stop(JobScheduler.scala:95) - locked <0x00000007b5d8ced8> (a org.apache.spark.streaming.scheduler.JobScheduler) at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:687) "JobGenerator" #67 daemon prio=5 os_prio=31 tid=0x00007fd35c3b9800 nid=0x9f03 waiting for monitor entry [0x0000000139e4a000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.spark.streaming.scheduler.JobGenerator.shouldCheckpoint$lzycompute(JobGenerator.scala:63) - waiting to lock <0x00000007b5d8cea0> (a org.apache.spark.streaming.scheduler.JobGenerator) at org.apache.spark.streaming.scheduler.JobGenerator.shouldCheckpoint(JobGenerator.scala:63) at org.apache.spark.streaming.scheduler.JobGenerator.doCheckpoint(JobGenerator.scala:290) at org.apache.spark.streaming.scheduler.JobGenerator.org$apache$spark$streaming$scheduler$JobGenerator$$processEvent(JobGenerator.scala:182) at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:83) at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:82) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) ``` I can use this patch to produce this deadlock: https://github.com/zsxwing/spark/commit/8a88f28d1331003a65fabef48ae3d22a7c21f05f And a timeout build in Jenkins due to this deadlock: https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1654/ This PR initializes `checkpointWriter` before `eventLoop` uses it to avoid this deadlock. Author: zsxwing <zsxwing@gmail.com> Closes #8326 from zsxwing/SPARK-10125.
* [SPARK-9812] [STREAMING] Fix Python 3 compatibility issue in PySpark ↵zsxwing2015-08-198-14/+23
| | | | | | | | | | | | | Streaming and some docs This PR includes the following fixes: 1. Use `range` instead of `xrange` in `queue_stream.py` to support Python 3. 2. Fix the issue that `utf8_decoder` will return `bytes` rather than `str` when receiving an empty `bytes` in Python 3. 3. Fix the commands in docs so that the user can copy them directly to the command line. The previous commands was broken in the middle of a path, so when copying to the command line, the path would be split to two parts by the extra spaces, which forces the user to fix it manually. Author: zsxwing <zsxwing@gmail.com> Closes #8315 from zsxwing/SPARK-9812.
* [SPARK-9242] [SQL] Audit UDAF interface.Reynold Xin2015-08-1918-349/+386
| | | | | | | | | | | | | | | | | | | A few minor changes: 1. Improved documentation 2. Rename apply(distinct....) to distinct. 3. Changed MutableAggregationBuffer from a trait to an abstract class. 4. Renamed returnDataType to dataType to be more consistent with other expressions. And unrelated to UDAFs: 1. Renamed file names in expressions to use suffix "Expressions" to be more consistent. 2. Moved regexp related expressions out to its own file. 3. Renamed StringComparison => StringPredicate. Author: Reynold Xin <rxin@databricks.com> Closes #8321 from rxin/SPARK-9242.
* [SPARK-10035] [SQL] Parquet filters does not process EqualNullSafe filter.hyukjinkwon2015-08-202-139/+37
| | | | | | | | | | | | | | | | | | | As I talked with Lian, 1. I added EquelNullSafe to ParquetFilters - It uses the same equality comparison filter with EqualTo since the Parquet filter performs actually null-safe equality comparison. 2. Updated the test code (ParquetFilterSuite) - Convert catalyst.Expression to sources.Filter - Removed Cast since only Literal is picked up as a proper Filter in DataSourceStrategy - Added EquelNullSafe comparison 3. Removed deprecated createFilter for catalyst.Expression Author: hyukjinkwon <gurwls223@gmail.com> Author: 권혁진 <gurwls223@gmail.com> Closes #8275 from HyukjinKwon/master.
* [SPARK-9895] User Guide for RFormula Feature TransformerEric Liang2015-08-192-2/+110
| | | | | | | | mengxr Author: Eric Liang <ekl@databricks.com> Closes #8293 from ericl/docs-2.
* [SPARK-6489] [SQL] add column pruning for GenerateWenchen Fan2015-08-193-2/+100
| | | | | | | | This PR takes over https://github.com/apache/spark/pull/5358 Author: Wenchen Fan <cloud0fan@outlook.com> Closes #8268 from cloud-fan/6489.
* [SPARK-10119] [CORE] Fix isDynamicAllocationEnabled when config is ↵Marcelo Vanzin2015-08-192-1/+15
| | | | | | | | expliticly disabled. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #8316 from vanzin/SPARK-10119.
* [SPARK-10083] [SQL] CaseWhen should support type coercion of DecimalType and ↵Daoyuan Wang2015-08-192-2/+13
| | | | | | | | | | | | FractionalType create t1 (a decimal(7, 2), b long); select case when 1=1 then a else 1.0 end from t1; select case when 1=1 then a else b end from t1; Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #8270 from adrian-wang/casewhenfractional.
* [SPARK-9899] [SQL] Disables customized output committer when speculation is onCheng Lian2015-08-192-1/+49
| | | | | | | | | | | | Speculation hates direct output committer, as there are multiple corner cases that may cause data corruption and/or data loss. Please see this [PR comment] [1] for more details. [1]: https://github.com/apache/spark/pull/8191#issuecomment-131598385 Author: Cheng Lian <lian@databricks.com> Closes #8317 from liancheng/spark-9899/speculation-hates-direct-output-committer.
* [SPARK-10090] [SQL] fix decimal scale of divisionDavies Liu2015-08-196-31/+157
| | | | | | | | We should rounding the result of multiply/division of decimal to expected precision/scale, also check overflow. Author: Davies Liu <davies@databricks.com> Closes #8287 from davies/decimal_division.
* [SPARK-9627] [SQL] Stops using Scala runtime reflection in DictionaryEncodingCheng Lian2015-08-192-12/+4
| | | | | | | | | | `DictionaryEncoding` uses Scala runtime reflection to avoid boxing costs while building the directory array. However, this code path may hit [SI-6240] [1] and throw exception. [1]: https://issues.scala-lang.org/browse/SI-6240 Author: Cheng Lian <lian@databricks.com> Closes #8306 from liancheng/spark-9627/in-memory-cache-scala-reflection.
* [SPARK-10073] [SQL] Python withColumn should replace the old columnDavies Liu2015-08-193-7/+12
| | | | | | | | | | DataFrame.withColumn in Python should be consistent with the Scala one (replacing the existing column that has the same name). cc marmbrus Author: Davies Liu <davies@databricks.com> Closes #8300 from davies/with_column.
* [SPARK-10107] [SQL] fix NPE in format_numberDavies Liu2015-08-192-3/+3
| | | | | | Author: Davies Liu <davies@databricks.com> Closes #8305 from davies/format_number.
* [SPARK-8889] [CORE] Fix for OOM for graph creationJoshi2015-08-192-11/+51
| | | | | | | | | Fix for OOM for graph creation Author: Joshi <rekhajoshm@gmail.com> Author: Rekha Joshi <rekhajoshm@gmail.com> Closes #7602 from rekhajoshm/SPARK-8889.
* [SPARK-8918] [MLLIB] [DOC] Add @since tags to mllib.clusteringXiangrui Meng2015-08-199-52/+338
| | | | | | | | | | | | This continues the work from #8256. I removed `since` tags from private/protected/local methods/variables (see https://github.com/apache/spark/commit/72fdeb64630470f6f46cf3eed8ffbfe83a7c4659). MechCoder Closes #8256 Author: Xiangrui Meng <meng@databricks.com> Author: Xiaoqing Wang <spark445@126.com> Author: MechCoder <manojkumarsivaraj334@gmail.com> Closes #8288 from mengxr/SPARK-8918.
* [SPARK-10106] [SPARKR] Add `ifelse` Column function to SparkRYu ISHIKAWA2015-08-193-1/+22
| | | | | | | | | ### JIRA [[SPARK-10106] Add `ifelse` Column function to SparkR - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-10106) Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Closes #8303 from yu-iskw/SPARK-10106.
* [SPARK-10097] Adds `shouldMaximize` flag to `ml.evaluation.Evaluator`Feynman Liang2015-08-1910-22/+52
| | | | | | | | | | | | | Previously, users of evaluator (`CrossValidator` and `TrainValidationSplit`) would only maximize the metric in evaluator, leading to a hacky solution which negated metrics to be minimized and caused erroneous negative values to be reported to the user. This PR adds a `isLargerBetter` attribute to the `Evaluator` base class, instructing users of `Evaluator` on whether the chosen metric should be maximized or minimized. CC jkbradley Author: Feynman Liang <fliang@databricks.com> Author: Joseph K. Bradley <joseph@databricks.com> Closes #8290 from feynmanliang/SPARK-10097.
* [SPARK-9833] [YARN] Add options to disable delegation token retrieval.Marcelo Vanzin2015-08-192-6/+33
| | | | | | | | | | This allows skipping the code that tries to talk to Hive and HBase to fetch delegation tokens, in case that somehow conflicts with the application being run. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #8134 from vanzin/SPARK-9833.
* [SPARK-9856] [SPARKR] Add expression functions into SparkR whose params are ↵Yu ISHIKAWA2015-08-195-6/+649
| | | | | | | | | | | | | complicated I added lots of Column functinos into SparkR. And I also added `rand(seed: Int)` and `randn(seed: Int)` in Scala. Since we need such APIs for R integer type. ### JIRA [[SPARK-9856] Add expression functions into SparkR whose params are complicated - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-9856) Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Closes #8264 from yu-iskw/SPARK-9856-3.
* [SPARK-5754] [YARN] Spark/Yarn/Windows driver/executor escaping FixCarsten Blank2015-08-194-12/+75
| | | | | | | | | | | | | | | | | This is my retry to suggest a fix for using Spark on Yarn on Windows. The former request lacked coding style which I hope to have learned to do better, and wasn't a true solution as I didn't really understand where the problem came from. Albeit being still a bit obscure, I can name the "players" and have come up with a better explaination of why I am suggesting this fix. I also used vanzin and srowen input to *try* to give a more elegant solution. I am not so sure if that worked out though. I still hope that this PR is a lot more useful than the last. Also do I hope that this is a _solution_ to the problem that Spark doesn't work on Yarn on Windows. With these changes it works (and I can also explain why!). I still believe that a Unit Test should be included, kind of like the one I committed the last time. But that was premature, as I want to get the principal 'Go' from vanzin and srowen. Thanks for your time both of you. Author: Carsten Blank <blank@cncengine.com> Author: cbvoxel <blank@cncengine.com> Closes #8053 from cbvoxel/master.
* [SPARK-10084] [MLLIB] [DOC] Add Python example for mllib FP-growth user guideYanbo Liang2015-08-191-23/+50
| | | | | | | | | 1, Add Python example for mllib FP-growth user guide. 2, Correct mistakes of Scala and Java examples. Author: Yanbo Liang <ybliang8@gmail.com> Closes #8279 from yanboliang/spark-10084.
* [SPARK-10060] [ML] [DOC] spark.ml DecisionTree user guideJoseph K. Bradley2015-08-195-13/+519
| | | | | | | | | | | | New user guide section ml-decision-tree.md, including code examples. I have run all examples, including the Java ones. CC: manishamde yanboliang mengxr Author: Joseph K. Bradley <joseph@databricks.com> Closes #8244 from jkbradley/ml-dt-docs.
* [SPARK-8949] Print warnings when using preferred locations featureHan JU2015-08-191-0/+5
| | | | | | | | | | | | Add warnings according to SPARK-8949 in `SparkContext` - warnings in scaladoc - log warnings when preferred locations feature is used through `SparkContext`'s constructor However I didn't found any documentation reference of this feature. Please direct me if you know any reference to this feature. Author: Han JU <ju.han.felix@gmail.com> Closes #7874 from darkjh/SPARK-8949.
* [SPARK-9977] [DOCS] Update documentation for StringIndexerlewuathe2015-08-191-1/+5
| | | | | | | | | By using `StringIndexer`, we can obtain indexed label on new column. So a following estimator should use this new column through pipeline if it wants to use string indexed label. I think it is better to make it explicit on documentation. Author: lewuathe <lewuathe@me.com> Closes #8205 from Lewuathe/SPARK-9977.
* [DOCS] [SQL] [PYSPARK] Fix typo in ntile functionMoussa Taifi2015-08-191-1/+1
| | | | | | | | Fix typo in ntile function. Author: Moussa Taifi <moutai10@gmail.com> Closes #8261 from moutai/patch-2.
* [SPARK-10070] [DOCS] Remove Guava dependencies in user guidesSean Owen2015-08-192-35/+38
| | | | | | | | | | | | `Lists.newArrayList` -> `Arrays.asList` CC jkbradley feynmanliang Anybody into replacing usages of `Lists.newArrayList` in the examples / source code too? this method isn't useful in Java 7 and beyond. Author: Sean Owen <sowen@cloudera.com> Closes #8272 from srowen/SPARK-10070.
* Fix Broken LinkBill Chambers2015-08-191-1/+1
| | | | | | | | Link was broken because it included tick marks. Author: Bill Chambers <wchambers@ischool.berkeley.edu> Closes #8302 from anabranch/patch-1.
* [SPARK-9967] [SPARK-10099] [STREAMING] Renamed conf ↵Tathagata Das2015-08-184-8/+8
| | | | | | | | | | | | spark.streaming.backpressure.{enable-->enabled} and fixed deprecated annotations Small changes - Renamed conf spark.streaming.backpressure.{enable --> enabled} - Change Java Deprecated annotations to Scala deprecated annotation with more information. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8299 from tdas/SPARK-9967.
* [SPARK-9952] Fix N^2 loop when DAGScheduler.getPreferredLocsInternal ↵Josh Rosen2015-08-184-16/+18
| | | | | | | | | | | | | | | | | | | | accesses cacheLocs In Scala, `Seq.fill` always seems to return a List. Accessing a list by index is an O(N) operation. Thus, the following code will be really slow (~10 seconds on my machine): ```scala val numItems = 100000 val s = Seq.fill(numItems)(1) for (i <- 0 until numItems) s(i) ``` It turns out that we had a loop like this in DAGScheduler code, although it's a little tricky to spot. In `getPreferredLocsInternal`, there's a call to `getCacheLocs(rdd)(partition)`. The `getCacheLocs` call returns a Seq. If this Seq is a List and the RDD contains many partitions, then indexing into this list will cost O(partitions). Thus, when we loop over our tasks to compute their individual preferred locations we implicitly perform an N^2 loop, reducing scheduling throughput. This patch fixes this by replacing `Seq` with `Array`. Author: Josh Rosen <joshrosen@databricks.com> Closes #8178 from JoshRosen/dagscheduler-perf.
* [SPARK-9508] GraphX Pregel docs update with new Pregel codeAlexander Ulanov2015-08-181-10/+8
| | | | | | | | SPARK-9436 simplifies the Pregel code. graphx-programming-guide needs to be modified accordingly since it lists the old Pregel code Author: Alexander Ulanov <nashb@yandex.ru> Closes #7831 from avulanov/SPARK-9508-pregel-doc2.
* [SPARK-9705] [DOC] fix docs about Python versionDavies Liu2015-08-182-3/+15
| | | | | | | | cc JoshRosen Author: Davies Liu <davies@databricks.com> Closes #8245 from davies/python_doc.
* [SPARK-10093] [SPARK-10096] [SQL] Avoid transformation on executors & fix ↵Reynold Xin2015-08-184-7/+68
| | | | | | | | | | | | | | | | | | UDFs on complex types This is kind of a weird case, but given a sufficiently complex query plan (in this case a TungstenProject with an Exchange underneath), we could have NPEs on the executors due to the time when we were calling transformAllExpressions In general we should ensure that all transformations occur on the driver and not on the executors. Some reasons for avoid executor side transformations include: * (this case) Some operator constructors require state such as access to the Spark/SQL conf so doing a makeCopy on the executor can fail. * (unrelated reason for avoid executor transformations) ExprIds are calculated using an atomic integer, so you can violate their uniqueness constraint by constructing them anywhere other than the driver. This subsumes #8285. Author: Reynold Xin <rxin@databricks.com> Author: Michael Armbrust <michael@databricks.com> Closes #8295 from rxin/SPARK-10096.
* [SPARK-10095] [SQL] use public API of BigIntegerDavies Liu2015-08-183-45/+11
| | | | | | | | | | | | In UnsafeRow, we use the private field of BigInteger for better performance, but it actually didn't contribute much (3% in one benchmark) to end-to-end runtime, and make it not portable (may fail on other JVM implementations). So we should use the public API instead. cc rxin Author: Davies Liu <davies@databricks.com> Closes #8286 from davies/portable_decimal.
* [SPARK-10075] [SPARKR] Add `when` expressino function in SparkRYu ISHIKAWA2015-08-185-0/+45
| | | | | | | | | | | | | | | - Add `when` and `otherwise` as `Column` methods - Add `When` as an expression function - Add `%otherwise%` infix as an alias of `otherwise` Since R doesn't support a feature like method chaining, `otherwise(when(condition, value), value)` style is a little annoying for me. If `%otherwise%` looks strange for shivaram, I can remove it. What do you think? ### JIRA [[SPARK-10075] Add `when` expressino function in SparkR - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-10075) Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Closes #8266 from yu-iskw/SPARK-10075.
* [SPARK-9939] [SQL] Resorts to Java process API in CliSuite, ↵Cheng Lian2015-08-195-91/+149
| | | | | | | | | | | | | | HiveSparkSubmitSuite and HiveThriftServer2 test suites Scala process API has a known bug ([SI-8768] [1]), which may be the reason why several test suites which fork sub-processes are flaky. This PR replaces Scala process API with Java process API in `CliSuite`, `HiveSparkSubmitSuite`, and `HiveThriftServer2` related test suites to see whether it fix these flaky tests. [1]: https://issues.scala-lang.org/browse/SI-8768 Author: Cheng Lian <lian@databricks.com> Closes #8168 from liancheng/spark-9939/use-java-process-api.
* [SPARK-10102] [STREAMING] Fix a race condition that startReceiver may happen ↵zsxwing2015-08-181-3/+8
| | | | | | | | | | | | | | before setting trackerState to Started Test failure: https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.4,label=spark-test/3305/testReport/junit/org.apache.spark.streaming/StreamingContextSuite/stop_gracefully/ There is a race condition that setting `trackerState` to `Started` could happen after calling `startReceiver`. Then `startReceiver` won't start the receivers because it uses `! isTrackerStarted` to check if ReceiverTracker is stopping or stopped. But actually, `trackerState` is `Initialized` and will be changed to `Started` soon. Therefore, we should use `isTrackerStopping || isTrackerStopped`. Author: zsxwing <zsxwing@gmail.com> Closes #8294 from zsxwing/SPARK-9504.
* [SPARK-10072] [STREAMING] BlockGenerator can deadlock when the queue of ↵Tathagata Das2015-08-181-10/+19
| | | | | | | | | | | | generate blocks fills up to capacity Generated blocks are inserted into an ArrayBlockingQueue, and another thread pulls stuff from the ArrayBlockingQueue and pushes it into BlockManager. Now if that queue fills up to capacity (default is 10 blocks), then the inserting into queue (done in the function updateCurrentBuffer) get blocked inside a synchronized block. However, the thread that is pulling blocks from the queue uses the same lock to check the current (active or stopped) while pulling from the queue. Since the block generating threads is blocked (as the queue is full) on the lock, this thread that is supposed to drain the queue gets blocked. Ergo, deadlock. Solution: Moved blocking call to ArrayBlockingQueue outside the synchronized to prevent deadlock. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8257 from tdas/SPARK-10072.
* [SPARKR] [MINOR] Get rid of a long line warningYu ISHIKAWA2015-08-181-1/+3
| | | | | | | | | | | | ``` R/functions.R:74:1: style: lines should not be more than 100 characters. jc <- callJStatic("org.apache.spark.sql.functions", "lit", ifelse(class(x) == "Column", xjc, x)) ^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ ``` Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Closes #8297 from yu-iskw/minor-lint-r.