Commit graph

15074 commits

Author SHA1 Message Date
Daoyuan Wang 5d80fac58f [SPARK-11624][SPARK-11972][SQL] fix commands that need hive to exec
In SparkSQLCLI, we have created a `CliSessionState`, but then we call `SparkSQLEnv.init()`, which will start another `SessionState`. This would lead to exception because `processCmd` need to get the `CliSessionState` instance by calling `SessionState.get()`, but the return value would be a instance of `SessionState`. See the exception below.

spark-sql> !echo "test";
Exception in thread "main" java.lang.ClassCastException: org.apache.hadoop.hive.ql.session.SessionState cannot be cast to org.apache.hadoop.hive.cli.CliSessionState
	at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:112)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:301)
	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:242)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:606)
	at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:691)
	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:205)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:120)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #9589 from adrian-wang/clicommand.
2016-02-22 18:13:32 -08:00
Shixiong Zhu a11b399519 [SPARK-13298][CORE][UI] Escape "label" to avoid DAG being broken by some special character
## What changes were proposed in this pull request?

When there are some special characters (e.g., `"`, `\`) in `label`, DAG will be broken. This patch just escapes `label` to avoid DAG being broken by some special characters

## How was the this patch tested?

Jenkins tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11309 from zsxwing/SPARK-13298.
2016-02-22 17:42:30 -08:00
Narine Kokhlikyan 33ef3aa7ea [SPARK-13295][ ML, MLLIB ] AFTSurvivalRegression.AFTAggregator improvements - avoid creating new instances of arrays/vectors for each record
As also mentioned/marked by TODO in AFTAggregator.AFTAggregator.add(data: AFTPoint) method a new array is being created for intercept value and it is being concatenated
with another array which contains the betas, the resulted Array is being converted into a Dense vector which in its turn is being converted into breeze vector.
This is expensive and not necessarily beautiful.

I've tried to solve above mentioned problem by simple algebraic decompositions - keeping and treating intercept independently.

Please let me know what do you think and if you have any questions.

Thanks,
Narine

Author: Narine Kokhlikyan <narine.kokhlikyan@gmail.com>

Closes #11179 from NarineK/survivaloptim.
2016-02-22 17:26:32 -08:00
Devaraj K 02b1fefffb [SPARK-13012][DOCUMENTATION] Replace example code in ml-guide.md using include_example
Replaced example code in ml-guide.md using include_example

Author: Devaraj K <devaraj@apache.org>

Closes #11053 from devaraj-kavali/SPARK-13012.
2016-02-22 17:21:37 -08:00
Devaraj K 9f410871ca [SPARK-13016][DOCUMENTATION] Replace example code in mllib-dimensionality-reduction.md using include_example
Replaced example example code in mllib-dimensionality-reduction.md using
include_example

Author: Devaraj K <devaraj@apache.org>

Closes #11132 from devaraj-kavali/SPARK-13016.
2016-02-22 17:16:56 -08:00
Xiu Guo 2063781840 [SPARK-13422][SQL] Use HashedRelation instead of HashSet in Left Semi Joins
Use the HashedRelation which is a more optimized datastructure and reduce code complexity

Author: Xiu Guo <xguo27@gmail.com>

Closes #11291 from xguo27/SPARK-13422.
2016-02-22 16:34:02 -08:00
Michael Armbrust 173aa949c3 [SPARK-12546][SQL] Change default number of open parquet files
A common problem that users encounter with Spark 1.6.0 is that writing to a partitioned parquet table OOMs.  The root cause is that parquet allocates a significant amount of memory that is not accounted for by our own mechanisms.  As a workaround, we can ensure that only a single file is open per task unless the user explicitly asks for more.

Author: Michael Armbrust <michael@databricks.com>

Closes #11308 from marmbrus/parquetWriteOOM.
2016-02-22 15:27:29 -08:00
Reynold Xin 4a91806a45 [SPARK-13413] Remove SparkContext.metricsSystem
## What changes were proposed in this pull request?

This patch removes SparkContext.metricsSystem. SparkContext.metricsSystem returns MetricsSystem, which is a private class. I think it was added by accident.

In addition, I also removed an unused private[spark] method schedulerBackend setter.

## How was the this patch tested?

N/A.

Author: Reynold Xin <rxin@databricks.com>

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

Closes #11282 from rxin/SPARK-13413.
2016-02-22 14:01:35 -08:00
Timothy Chen 00461bb911 [SPARK-10749][MESOS] Support multiple roles with mesos cluster mode.
Currently the Mesos cluster dispatcher is not using offers from multiple roles correctly, as it simply aggregates all the offers resource values into one, but doesn't apply them correctly before calling the driver as Mesos needs the resources from the offers to be specified which role it originally belongs to. Multiple roles is already supported with fine/coarse grain scheduler, so porting that logic here to the cluster scheduler.

https://issues.apache.org/jira/browse/SPARK-10749

Author: Timothy Chen <tnachen@gmail.com>

Closes #8872 from tnachen/cluster_multi_roles.
2016-02-22 11:11:33 -08:00
Yanbo Liang 40e6d40fe7 [SPARK-13334][ML] ML KMeansModel / BisectingKMeansModel / QuantileDiscretizer should set parent
ML ```KMeansModel / BisectingKMeansModel / QuantileDiscretizer``` should set parent.

cc mengxr

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #11214 from yanboliang/spark-13334.
2016-02-22 12:59:50 +02:00
Bryan Cutler e298ac91e3 [SPARK-12632][PYSPARK][DOC] PySpark fpm and als parameter desc to consistent format
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent.  This is for the fpm and recommendation modules.

Closes #10602
Closes #10897

Author: Bryan Cutler <cutlerb@gmail.com>
Author: somideshmukh <somilde@us.ibm.com>

Closes #11186 from BryanCutler/param-desc-consistent-fpmrecc-SPARK-12632.
2016-02-22 12:48:37 +02:00
Dongjoon Hyun 024482bf51 [MINOR][DOCS] Fix all typos in markdown files of doc and similar patterns in other comments
## What changes were proposed in this pull request?

This PR tries to fix all typos in all markdown files under `docs` module,
and fixes similar typos in other comments, too.

## How was the this patch tested?

manual tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11300 from dongjoon-hyun/minor_fix_typos.
2016-02-22 09:52:07 +00:00
Holden Karau 1b144455b6 [SPARK-13399][STREAMING] Fix checkpointsuite type erasure warnings
## What changes were proposed in this pull request?

Change the checkpointsuite getting the outputstreams to explicitly be unchecked on the generic type so as to avoid the warnings. This only impacts test code.

Alternatively we could encode the type tag in the TestOutputStreamWithPartitions and filter the type tag as well - but this is unnecessary since multiple testoutputstreams are not registered and the previous code was not actually checking this type.

## How was the this patch tested?

unit tests (streaming/testOnly org.apache.spark.streaming.CheckpointSuite)

Author: Holden Karau <holden@us.ibm.com>

Closes #11286 from holdenk/SPARK-13399-checkpointsuite-type-erasure.
2016-02-22 09:50:51 +00:00
Yong Gang Cao ef1047fca7 [SPARK-12153][SPARK-7617][MLLIB] add support of arbitrary length sentence and other tuning for Word2Vec
add support of arbitrary length sentence by using the nature representation of sentences in the input.

add new similarity functions and add normalization option for distances in synonym finding
add new accessor for internal structure(the vocabulary and wordindex) for convenience

need instructions about how to set value for the Since annotation for newly added public functions. 1.5.3?

jira link: https://issues.apache.org/jira/browse/SPARK-12153

Author: Yong Gang Cao <ygcao@amazon.com>
Author: Yong-Gang Cao <ygcao@users.noreply.github.com>

Closes #10152 from ygcao/improvementForSentenceBoundary.
2016-02-22 09:47:36 +00:00
Huaxin Gao 8f35d3eac9 [SPARK-13186][STREAMING] migrate away from SynchronizedMap
trait SynchronizedMap in package mutable is deprecated: Synchronization via traits is deprecated as it is inherently unreliable. Change to java.util.concurrent.ConcurrentHashMap instead.

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #11250 from huaxingao/spark__13186.
2016-02-22 09:44:32 +00:00
jerryshao 39ff154570 [SPARK-13426][CORE] Remove the support of SIMR
## What changes were proposed in this pull request?

This PR removes the support of SIMR, since SIMR is not actively used and maintained for a long time, also is not supported from `SparkSubmit`, so here propose to remove it.

## How was the this patch tested?

This patch is tested locally by running unit tests.

Author: jerryshao <sshao@hortonworks.com>

Closes #11296 from jerryshao/SPARK-13426.
2016-02-22 00:57:10 -08:00
Yanbo Liang 8a4ed78869 [SPARK-13379][MLLIB] Fix MLlib LogisticRegressionWithLBFGS set regularization incorrectly
## What changes were proposed in this pull request?
Fix MLlib LogisticRegressionWithLBFGS regularization map as:
```SquaredL2Updater``` -> ```elasticNetParam = 0.0```
```L1Updater``` -> ```elasticNetParam = 1.0```
cc dbtsai
## How was the this patch tested?
unit tests

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #11258 from yanboliang/spark-13379.
2016-02-21 20:20:41 -08:00
Reynold Xin 9bf6a926a1 [HOTFIX] Fix compilation break 2016-02-21 19:37:35 -08:00
hyukjinkwon 819b0ea029 [SPARK-13381][SQL] Support for loading CSV with a single function call
https://issues.apache.org/jira/browse/SPARK-13381

This PR adds the support to load CSV data directly by a single call with given paths.

Also, I corrected this to refer all paths rather than the first path in schema inference, which JSON datasource dose.

Several unitests were added for each functionality.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11262 from HyukjinKwon/SPARK-13381.
2016-02-21 19:11:03 -08:00
Liang-Chi Hsieh 55d6fdf22d [SPARK-13321][SQL] Support nested UNION in parser
JIRA: https://issues.apache.org/jira/browse/SPARK-13321

The following SQL can not be parsed with current parser:

    SELECT  `u_1`.`id` FROM (((SELECT  `t0`.`id` FROM `default`.`t0`) UNION ALL (SELECT  `t0`.`id` FROM `default`.`t0`)) UNION ALL (SELECT  `t0`.`id` FROM `default`.`t0`)) AS u_1

We should fix it.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #11204 from viirya/nested-union.
2016-02-21 19:10:17 -08:00
Robin East 3d79f1065c [SPARK-3650][GRAPHX] Triangle Count handles reverse edges incorrectly
jegonzal ankurdave please could you review

## What changes were proposed in this pull request?

Reworking of jegonzal PR #2495 to address the issue identified in SPARK-3650. Code amended to use the convertToCanonicalEdges method.

## How was the this patch tested?

Patch was tested using the unit tests created in PR #2495

Author: Robin East <robin.east@xense.co.uk>
Author: Joseph E. Gonzalez <joseph.e.gonzalez@gmail.com>

Closes #11290 from insidedctm/spark-3650.
2016-02-21 17:07:17 -08:00
Franklyn D'souza 0f90f4e6ac [SPARK-13410][SQL] Support unionAll for DataFrames with UDT columns.
## What changes were proposed in this pull request?

This PR adds equality operators to UDT classes so that they can be correctly tested for dataType equality during union operations.

This was previously causing `"AnalysisException: u"unresolved operator 'Union;""` when trying to unionAll two dataframes with UDT columns as below.

```
from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
from pyspark.sql import types

schema = types.StructType([types.StructField("point", PythonOnlyUDT(), True)])

a = sqlCtx.createDataFrame([[PythonOnlyPoint(1.0, 2.0)]], schema)
b = sqlCtx.createDataFrame([[PythonOnlyPoint(3.0, 4.0)]], schema)

c = a.unionAll(b)
```

## How was the this patch tested?

Tested using two unit tests in sql/test.py and the DataFrameSuite.

Additional information here : https://issues.apache.org/jira/browse/SPARK-13410

Author: Franklyn D'souza <franklynd@gmail.com>

Closes #11279 from damnMeddlingKid/udt-union-all.
2016-02-21 16:58:17 -08:00
Shixiong Zhu 0cbadf28c9 [SPARK-13271][SQL] Better error message if 'path' is not specified
Improved the error message as per discussion in https://github.com/apache/spark/pull/11034#discussion_r52111238. Also made `path` and `metadataPath` in FileStreamSource case insensitive.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11154 from zsxwing/path.
2016-02-21 15:34:39 -08:00
Shixiong Zhu 76bd98d914 [SPARK-13405][STREAMING][TESTS] Make sure no messages leak to the next test
## What changes were proposed in this pull request?

Fixed the test failure `org.apache.spark.sql.util.ContinuousQueryListenerSuite.event ordering`: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/202/testReport/junit/org.apache.spark.sql.util/ContinuousQueryListenerSuite/event_ordering/

```
      org.scalatest.exceptions.TestFailedException:
Assert failed: : null equaled null onQueryTerminated called before onQueryStarted
org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
	org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
	org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)
	org.apache.spark.sql.util.ContinuousQueryListenerSuite$QueryStatusCollector$$anonfun$onQueryTerminated$1.apply$mcV$sp(ContinuousQueryListenerSuite.scala:204)
	org.scalatest.concurrent.AsyncAssertions$Waiter.apply(AsyncAssertions.scala:349)
	org.apache.spark.sql.util.ContinuousQueryListenerSuite$QueryStatusCollector.onQueryTerminated(ContinuousQueryListenerSuite.scala:203)
	org.apache.spark.sql.execution.streaming.ContinuousQueryListenerBus.doPostEvent(ContinuousQueryListenerBus.scala:67)
	org.apache.spark.sql.execution.streaming.ContinuousQueryListenerBus.doPostEvent(ContinuousQueryListenerBus.scala:32)
	org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:63)
	org.apache.spark.sql.execution.streaming.ContinuousQueryListenerBus.postToAll(ContinuousQueryListenerBus.scala:32)
```

In the previous codes, when the test `adding and removing listener` finishes, there may be still some QueryTerminated events in the listener bus queue. Then when `event ordering` starts to run, it may see these events and throw the above exception.

This PR just added `waitUntilEmpty` in `after` to make sure all events be consumed after each test.

## How was the this patch tested?

Jenkins tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11275 from zsxwing/SPARK-13405.
2016-02-21 15:32:49 -08:00
Dongjoon Hyun 03e62aa3f6 [MINOR][DOCS] Fix typos in configuration.md and hardware-provisioning.md
## What changes were proposed in this pull request?

This PR fixes some typos in the following documentation files.
 * `NOTICE`, `configuration.md`, and `hardware-provisioning.md`.

## How was the this patch tested?

manual tests

Author: Dongjoon Hyun <dongjoonapache.org>

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11289 from dongjoon-hyun/minor_fix_typos_notice_and_confdoc.
2016-02-21 15:27:07 -08:00
Andrew Or 6c3832b26e [SPARK-13080][SQL] Implement new Catalog API using Hive
## What changes were proposed in this pull request?

This is a step towards merging `SQLContext` and `HiveContext`. A new internal Catalog API was introduced in #10982 and extended in #11069. This patch introduces an implementation of this API using `HiveClient`, an existing interface to Hive. It also extends `HiveClient` with additional calls to Hive that are needed to complete the catalog implementation.

*Where should I start reviewing?* The new catalog introduced is `HiveCatalog`. This class is relatively simple because it just calls `HiveClientImpl`, where most of the new logic is. I would not start with `HiveClient`, `HiveQl`, or `HiveMetastoreCatalog`, which are modified mainly because of a refactor.

*Why is this patch so big?* I had to refactor HiveClient to remove an intermediate representation of databases, tables, partitions etc. After this refactor `CatalogTable` convert directly to and from `HiveTable` (etc.). Otherwise we would have to first convert `CatalogTable` to the intermediate representation and then convert that to HiveTable, which is messy.

The new class hierarchy is as follows:
```
org.apache.spark.sql.catalyst.catalog.Catalog
  - org.apache.spark.sql.catalyst.catalog.InMemoryCatalog
  - org.apache.spark.sql.hive.HiveCatalog
```

Note that, as of this patch, none of these classes are currently used anywhere yet. This will come in the future before the Spark 2.0 release.

## How was the this patch tested?
All existing unit tests, and HiveCatalogSuite that extends CatalogTestCases.

Author: Andrew Or <andrew@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #11293 from rxin/hive-catalog.
2016-02-21 15:00:24 -08:00
hyukjinkwon 7eb83fefd1 [SPARK-13137][SQL] NullPoingException in schema inference for CSV when the first line is empty
https://issues.apache.org/jira/browse/SPARK-13137

This PR adds a filter in schema inference so that it does not emit NullPointException.

Also, I removed `MAX_COMMENT_LINES_IN_HEADER `but instead used a monad chaining with `filter()` and `first()`.

Lastly, I simply added a newline rather than adding a new file for this so that this is covered with the original tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11023 from HyukjinKwon/SPARK-13137.
2016-02-21 13:21:59 -08:00
Herman van Hovell b6a873d6d4 [SPARK-13136][SQL] Create a dedicated Broadcast exchange operator
Quite a few Spark SQL join operators broadcast one side of the join to all nodes. The are a few problems with this:

- This conflates broadcasting (a data exchange) with joining. Data exchanges should be managed by a different operator.
- All these nodes implement their own (duplicate) broadcasting logic.
- Re-use of indices is quite hard.

This PR defines both a ```BroadcastDistribution``` and ```BroadcastPartitioning```, these contain a `BroadcastMode`. The `BroadcastMode` defines the way in which we transform the Array of `InternalRow`'s into an index. We currently support the following `BroadcastMode`'s:

- IdentityBroadcastMode: This broadcasts the rows in their original form.
- HashSetBroadcastMode: This applies a projection to the input rows, deduplicates these rows and broadcasts the resulting `Set`.
- HashedRelationBroadcastMode: This transforms the input rows into a `HashedRelation`, and broadcasts this index.

To match this distribution we implement a ```BroadcastExchange``` operator which will perform the broadcast for us, and have ```EnsureRequirements``` plan this operator. The old Exchange operator has been renamed into ShuffleExchange in order to clearly separate between Shuffled and Broadcasted exchanges. Finally the classes in Exchange.scala have been moved to a dedicated package.

cc rxin davies

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #11083 from hvanhovell/SPARK-13136.
2016-02-21 12:32:31 -08:00
Reynold Xin af441ddbd1 [SPARK-13306][SQL] Addendum to uncorrelated scalar subquery
## What changes were proposed in this pull request?
This pull request fixes some minor issues (documentation, test flakiness, test organization) with #11190, which was merged earlier tonight.

## How was the this patch tested?
unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11285 from rxin/subquery.
2016-02-21 12:27:02 -08:00
Reynold Xin 0947f0989b [SPARK-13420][SQL] Rename Subquery logical plan to SubqueryAlias
## What changes were proposed in this pull request?
This patch renames logical.Subquery to logical.SubqueryAlias, which is a more appropriate name for this operator (versus subqueries as expressions).

## How was the this patch tested?
Unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11288 from rxin/SPARK-13420.
2016-02-21 11:31:46 -08:00
Luciano Resende 1a340da8d7 [SPARK-13248][STREAMING] Remove deprecated Streaming APIs.
Remove deprecated Streaming APIs and adjust sample applications.

Author: Luciano Resende <lresende@apache.org>

Closes #11139 from lresende/streaming-deprecated-apis.
2016-02-21 16:27:56 +00:00
Cheng Lian d9efe63ecd [SPARK-12799] Simplify various string output for expressions
This PR introduces several major changes:

1. Replacing `Expression.prettyString` with `Expression.sql`

   The `prettyString` method is mostly an internal, developer faced facility for debugging purposes, and shouldn't be exposed to users.

1. Using SQL-like representation as column names for selected fields that are not named expression (back-ticks and double quotes should be removed)

   Before, we were using `prettyString` as column names when possible, and sometimes the result column names can be weird.  Here are several examples:

   Expression         | `prettyString` | `sql`      | Note
   ------------------ | -------------- | ---------- | ---------------
   `a && b`           | `a && b`       | `a AND b`  |
   `a.getField("f")`  | `a[f]`         | `a.f`      | `a` is a struct

1. Adding trait `NonSQLExpression` extending from `Expression` for expressions that don't have a SQL representation (e.g. Scala UDF/UDAF and Java/Scala object expressions used for encoders)

   `NonSQLExpression.sql` may return an arbitrary user facing string representation of the expression.

Author: Cheng Lian <lian@databricks.com>

Closes #10757 from liancheng/spark-12799.simplify-expression-string-methods.
2016-02-21 22:53:15 +08:00
Zheng RuiFeng d806ed3436 [SPARK-13416][GraphX] Add positive check for option 'numIter' in StronglyConnectedComponents
JIRA: https://issues.apache.org/jira/browse/SPARK-13416

## What changes were proposed in this pull request?

The output of StronglyConnectedComponents with numIter no greater than 1 may make no sense. So I just add require check in it.

## How was the this patch tested?

 unit tests passed

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #11284 from zhengruifeng/scccheck.
2016-02-21 00:53:15 -08:00
Davies Liu 7925071280 [SPARK-13306] [SQL] uncorrelated scalar subquery
A scalar subquery is a subquery that only generate single row and single column, could be used as part of expression. Uncorrelated scalar subquery means it does not has a reference to external table.

All the uncorrelated scalar subqueries will be executed during prepare() of SparkPlan.

The plans for query
```sql
select 1 + (select 2 + (select 3))
```
looks like this
```
== Parsed Logical Plan ==
'Project [unresolvedalias((1 + subquery#1),None)]
:- OneRowRelation$
+- 'Subquery subquery#1
   +- 'Project [unresolvedalias((2 + subquery#0),None)]
      :- OneRowRelation$
      +- 'Subquery subquery#0
         +- 'Project [unresolvedalias(3,None)]
            +- OneRowRelation$

== Analyzed Logical Plan ==
_c0: int
Project [(1 + subquery#1) AS _c0#4]
:- OneRowRelation$
+- Subquery subquery#1
   +- Project [(2 + subquery#0) AS _c0#3]
      :- OneRowRelation$
      +- Subquery subquery#0
         +- Project [3 AS _c0#2]
            +- OneRowRelation$

== Optimized Logical Plan ==
Project [(1 + subquery#1) AS _c0#4]
:- OneRowRelation$
+- Subquery subquery#1
   +- Project [(2 + subquery#0) AS _c0#3]
      :- OneRowRelation$
      +- Subquery subquery#0
         +- Project [3 AS _c0#2]
            +- OneRowRelation$

== Physical Plan ==
WholeStageCodegen
:  +- Project [(1 + subquery#1) AS _c0#4]
:     :- INPUT
:     +- Subquery subquery#1
:        +- WholeStageCodegen
:           :  +- Project [(2 + subquery#0) AS _c0#3]
:           :     :- INPUT
:           :     +- Subquery subquery#0
:           :        +- WholeStageCodegen
:           :           :  +- Project [3 AS _c0#2]
:           :           :     +- INPUT
:           :           +- Scan OneRowRelation[]
:           +- Scan OneRowRelation[]
+- Scan OneRowRelation[]
```

Author: Davies Liu <davies@databricks.com>

Closes #11190 from davies/scalar_subquery.
2016-02-20 21:01:51 -08:00
gatorsmile f88c641bc8 [SPARK-13310] [SQL] Resolve Missing Sorting Columns in Generate
```scala
// case 1: missing sort columns are resolvable if join is true
sql("SELECT explode(a) AS val, b FROM data WHERE b < 2 order by val, c")
// case 2: missing sort columns are not resolvable if join is false. Thus, issue an error message in this case
sql("SELECT explode(a) AS val FROM data order by val, c")
```

When sort columns are not in `Generate`, we can resolve them when `join` is equal to `true`. Still trying to add more test cases for the other `UnaryNode` types.

Could you review the changes? davies cloud-fan Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11198 from gatorsmile/missingInSort.
2016-02-20 13:53:23 -08:00
Timothy Chen a4a081d1df [SPARK-13414][MESOS] Allow multiple dispatchers to be launched.
## What changes were proposed in this pull request?

Users might want to start multiple mesos dispatchers, as each dispatcher can potentially be part of different roles and used for multi-tenancy.

To allow multiple Mesos dispatchers to be launched, we need to be able to specify a instance number when starting the dispatcher daemon.

## How was the this patch tested?

Manual testing

Author: Timothy Chen <tnachen@gmail.com>

Closes #11281 from tnachen/multiple_cluster_dispatchers.
2016-02-20 12:58:47 -08:00
Zheng RuiFeng 6ce7c481dc [SPARK-13386][GRAPHX] ConnectedComponents should support maxIteration option
JIRA: https://issues.apache.org/jira/browse/SPARK-13386

## What changes were proposed in this pull request?

add maxIteration option for ConnectedComponents algorithm

## How was the this patch tested?

unit tests passed

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #11268 from zhengruifeng/ccwithmax.
2016-02-20 12:24:10 -08:00
Holden Karau 9ca79c1ece [SPARK-13302][PYSPARK][TESTS] Move the temp file creation and cleanup outside of the doctests
Some of the new doctests in ml/clustering.py have a lot of setup code, move the setup code to the general test init to keep the doctest more example-style looking.
In part this is a follow up to https://github.com/apache/spark/pull/10999
Note that the same pattern is followed in regression & recommendation - might as well clean up all three at the same time.

Author: Holden Karau <holden@us.ibm.com>

Closes #11197 from holdenk/SPARK-13302-cleanup-doctests-in-ml-clustering.
2016-02-20 09:07:19 +00:00
Shixiong Zhu dfb2ae2f14 [SPARK-13408] [CORE] Ignore errors when it's already reported in JobWaiter
## What changes were proposed in this pull request?

`JobWaiter.taskSucceeded` will be called for each task. When `resultHandler` throws an exception, `taskSucceeded` will also throw it for each task. DAGScheduler just catches it and reports it like this:
```Scala
                  try {
                    job.listener.taskSucceeded(rt.outputId, event.result)
                  } catch {
                    case e: Exception =>
                      // TODO: Perhaps we want to mark the resultStage as failed?
                      job.listener.jobFailed(new SparkDriverExecutionException(e))
                  }
```
Therefore `JobWaiter.jobFailed` may be called multiple times.

So `JobWaiter.jobFailed` should use `Promise.tryFailure` instead of `Promise.failure` because the latter one doesn't support calling multiple times.

## How was the this patch tested?

Jenkins tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11280 from zsxwing/SPARK-13408.
2016-02-19 23:00:08 -08:00
Reynold Xin 6624a588c1 Revert "[SPARK-12567] [SQL] Add aes_{encrypt,decrypt} UDFs"
This reverts commit 4f9a664818.
2016-02-19 22:44:20 -08:00
Kai Jiang 4f9a664818 [SPARK-12567] [SQL] Add aes_{encrypt,decrypt} UDFs
Author: Kai Jiang <jiangkai@gmail.com>

Closes #10527 from vectorijk/spark-12567.
2016-02-19 22:28:47 -08:00
gatorsmile ec7a1d6e42 [SPARK-12594] [SQL] Outer Join Elimination by Filter Conditions
Conversion of outer joins, if the predicates in filter conditions can restrict the result sets so that all null-supplying rows are eliminated.

- `full outer` -> `inner` if both sides have such predicates
- `left outer` -> `inner` if the right side has such predicates
- `right outer` -> `inner` if the left side has such predicates
- `full outer` -> `left outer` if only the left side has such predicates
- `full outer` -> `right outer` if only the right side has such predicates

If applicable, this can greatly improve the performance, since outer join is much slower than inner join, full outer join is much slower than left/right outer join.

The original PR is https://github.com/apache/spark/pull/10542

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10567 from gatorsmile/outerJoinEliminationByFilterCond.
2016-02-19 22:27:10 -08:00
Josh Rosen 983fa2d620 [SPARK-13407] Guard against garbage-collected accumulators in TaskMetrics.fromAccumulatorUpdates
`TaskMetrics.fromAccumulatorUpdates()` can fail if accumulators have been garbage-collected on the driver. To guard against this, this patch introduces `ListenerTaskMetrics`, a subclass of `TaskMetrics` which is used only in `TaskMetrics.fromAccumulatorUpdates()` and which eliminates the need to access the original accumulators on the driver.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11276 from JoshRosen/accum-updates-fix.
2016-02-19 15:57:23 -08:00
Sameer Agarwal 091f6a7830 [SPARK-13091][SQL] Rewrite/Propagate constraints for Aliases
This PR adds support for rewriting constraints if there are aliases in the query plan. For e.g., if there is a query of form `SELECT a, a AS b`, any constraints on `a` now also apply to `b`.

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

cc marmbrus

Author: Sameer Agarwal <sameer@databricks.com>

Closes #11144 from sameeragarwal/alias.
2016-02-19 14:48:34 -08:00
Hossein 14844118b5 [SPARK-13261][SQL] Expose maxCharactersPerColumn as a user configurable option
This patch expose `maxCharactersPerColumn` and `maxColumns` to user in CSV data source.

Author: Hossein <hossein@databricks.com>

Closes #11147 from falaki/SPARK-13261.
2016-02-19 14:46:56 -08:00
Brandon Bradley dbb08cdd5a [SPARK-12966][SQL] ArrayType(DecimalType) support in Postgres JDBC
Fixes error `org.postgresql.util.PSQLException: Unable to find server array type for provided name decimal(38,18)`.

* Passes scale metadata to JDBC dialect for usage in type conversions.
* Removes unused length/scale/precision parameters from `createArrayOf` parameter `typeName` (for writing).
* Adds configurable precision and scale to Postgres `DecimalType` (for reading).
* Adds a new kind of test that verifies the schema written by `DataFrame.write.jdbc`.

Author: Brandon Bradley <bradleytastic@gmail.com>

Closes #10928 from blbradley/spark-12966.
2016-02-19 14:43:21 -08:00
Liang-Chi Hsieh c7c55637bf [SPARK-13384][SQL] Keep attribute qualifiers after dedup in Analyzer
JIRA: https://issues.apache.org/jira/browse/SPARK-13384

## What changes were proposed in this pull request?

When we de-duplicate attributes in Analyzer, we create new attributes. However, we don't keep original qualifiers. Some plans will be failed to analysed. We should keep original qualifiers in new attributes.

## How was the this patch tested?

Unit test is added.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #11261 from viirya/keep-attr-qualifiers.
2016-02-19 12:22:22 -08:00
Iulian Dragos 6915cc23b3 [MINOR][DOCS][MESOS] Clarify that Mesos version is a lower bound.
## What changes were proposed in this pull request?

Clarify that 0.21 is only a **minimum** requirement.

## How was the this patch tested?

It's a doc change, so no tests.

Author: Iulian Dragos <jaguarul@gmail.com>

Closes #11271 from dragos/patch-1.
2016-02-19 11:47:36 -08:00
Sean Owen fb7e21797e [SPARK-13339][DOCS] Clarify commutative / associative operator requirements for reduce, fold
Clarify that reduce functions need to be commutative, and fold functions do not

See https://github.com/apache/spark/pull/11091

Author: Sean Owen <sowen@cloudera.com>

Closes #11217 from srowen/SPARK-13339.
2016-02-19 10:26:38 +00:00
gatorsmile c776fce99b [SPARK-13380][SQL][DOCUMENT] Document Rand(seed) and Randn(seed) Return Indeterministic Results When Data Partitions are not fixed.
`rand` and `randn` functions with a `seed` argument are commonly used. Based on the common sense, the results of `rand` and `randn` should be deterministic if the `seed` parameter value is provided. For example, in MS SQL Server, it also has a function `rand`. Regarding the parameter `seed`, the description is like: ```Seed is an integer expression (tinyint, smallint, or int) that gives the seed value. If seed is not specified, the SQL Server Database Engine assigns a seed value at random. For a specified seed value, the result returned is always the same.```

Update: the current implementation is unable to generate deterministic results when the partitions are not fixed. This PR documents this issue in the function descriptions.

jkbradley hit an issue and provided an example in the following JIRA: https://issues.apache.org/jira/browse/SPARK-13333

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11232 from gatorsmile/randSeed.
2016-02-18 21:19:36 -08:00