Commit graph

17395 commits

Author SHA1 Message Date
petermaxlee a117afa7c2 [SPARK-17149][SQL] array.sql for testing array related functions
## What changes were proposed in this pull request?
This patch creates array.sql in SQLQueryTestSuite for testing array related functions, including:

- indexing
- array creation
- size
- array_contains
- sort_array

## How was this patch tested?
The patch itself is about adding tests.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14708 from petermaxlee/SPARK-17149.
2016-08-19 18:14:45 -07:00
Junyang Qian acac7a508a [SPARK-16443][SPARKR] Alternating Least Squares (ALS) wrapper
## What changes were proposed in this pull request?

Add Alternating Least Squares wrapper in SparkR. Unit tests have been updated.

## How was this patch tested?

SparkR unit tests.

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

![screen shot 2016-07-27 at 3 50 31 pm](https://cloud.githubusercontent.com/assets/15318264/17195347/f7a6352a-5411-11e6-8e21-61a48070192a.png)
![screen shot 2016-07-27 at 3 50 46 pm](https://cloud.githubusercontent.com/assets/15318264/17195348/f7a7d452-5411-11e6-845f-6d292283bc28.png)

Author: Junyang Qian <junyangq@databricks.com>

Closes #14384 from junyangq/SPARK-16443.
2016-08-19 14:24:09 -07:00
Sital Kedia cf0cce9036 [SPARK-17113] [SHUFFLE] Job failure due to Executor OOM in offheap mode
## What changes were proposed in this pull request?

This PR fixes executor OOM in offheap mode due to bug in Cooperative Memory Management for UnsafeExternSorter.  UnsafeExternalSorter was checking if memory page is being used by upstream by comparing the base object address of the current page with the base object address of upstream. However, in case of offheap memory allocation, the base object addresses are always null, so there was no spilling happening and eventually the operator would OOM.

Following is the stack trace this issue addresses -
java.lang.OutOfMemoryError: Unable to acquire 1220 bytes of memory, got 0
	at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120)
	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:341)
	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:362)
	at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:93)
	at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:170)

## How was this patch tested?

Tested by running the failing job.

Author: Sital Kedia <skedia@fb.com>

Closes #14693 from sitalkedia/fix_offheap_oom.
2016-08-19 11:27:30 -07:00
Kousuke Saruta 071eaaf9d2 [SPARK-11227][CORE] UnknownHostException can be thrown when NameNode HA is enabled.
## What changes were proposed in this pull request?

If the following conditions are satisfied, executors don't load properties in `hdfs-site.xml` and UnknownHostException can be thrown.

(1) NameNode HA is enabled
(2) spark.eventLogging is disabled or logging path is NOT on HDFS
(3) Using Standalone or Mesos for the cluster manager
(4) There are no code to load `HdfsCondition` class in the driver regardless of directly or indirectly.
(5) The tasks access to HDFS

(There might be some more conditions...)

For example, following code causes UnknownHostException when the conditions above are satisfied.
```
sc.textFile("<path on HDFS>").collect

```

```
java.lang.IllegalArgumentException: java.net.UnknownHostException: hacluster
	at org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:378)
	at org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:310)
	at org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:176)
	at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:678)
	at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:619)
	at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:149)
	at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2653)
	at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:92)
	at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2687)
	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2669)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:371)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:170)
	at org.apache.hadoop.mapred.JobConf.getWorkingDirectory(JobConf.java:656)
	at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:438)
	at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:411)
	at org.apache.spark.SparkContext$$anonfun$hadoopFile$1$$anonfun$32.apply(SparkContext.scala:986)
	at org.apache.spark.SparkContext$$anonfun$hadoopFile$1$$anonfun$32.apply(SparkContext.scala:986)
	at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:177)
	at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:177)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:177)
	at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:213)
	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:209)
	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:102)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
	at org.apache.spark.scheduler.Task.run(Task.scala:85)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
Caused by: java.net.UnknownHostException: hacluster
```

But following code doesn't cause the Exception because `textFile` method loads `HdfsConfiguration` indirectly.

```
sc.textFile("<path on HDFS>").collect
```

When a job includes some operations which access to HDFS, the object of `org.apache.hadoop.Configuration` is wrapped by `SerializableConfiguration`,  serialized and broadcasted from driver to executors and each executor deserialize the object with `loadDefaults` false so HDFS related properties should be set before broadcasted.

## How was this patch tested?
Tested manually on my standalone cluster.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #13738 from sarutak/SPARK-11227.
2016-08-19 10:11:25 -05:00
Alex Bozarth e98eb2146f [SPARK-16673][WEB UI] New Executor Page removed conditional for Logs and Thread Dump columns
## What changes were proposed in this pull request?

When #13670 switched `ExecutorsPage` to use JQuery DataTables it incidentally removed the conditional for the Logs and Thread Dump columns. I reimplemented the conditional display of the Logs and Thread dump columns as it was before the switch.

## How was this patch tested?

Manually tested and dev/run-tests

![both](https://cloud.githubusercontent.com/assets/13952758/17186879/da8dd1a8-53eb-11e6-8b0c-d0ff0156a9a7.png)
![dump](https://cloud.githubusercontent.com/assets/13952758/17186881/dab08a04-53eb-11e6-8b1c-50ffd0bf2ae8.png)
![logs](https://cloud.githubusercontent.com/assets/13952758/17186880/dab04d00-53eb-11e6-8754-68dd64d6d9f4.png)

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #14382 from ajbozarth/spark16673.
2016-08-19 10:04:20 -05:00
Reynold Xin 67e59d464f [SPARK-16994][SQL] Whitelist operators for predicate pushdown
## What changes were proposed in this pull request?
This patch changes predicate pushdown optimization rule (PushDownPredicate) from using a blacklist to a whitelist. That is to say, operators must be explicitly allowed. This approach is more future-proof: previously it was possible for us to introduce a new operator and then render the optimization rule incorrect.

This also fixes the bug that previously we allowed pushing filter beneath limit, which was incorrect. That is to say, before this patch, the optimizer would rewrite
```
select * from (select * from range(10) limit 5) where id > 3

to

select * from range(10) where id > 3 limit 5
```

## How was this patch tested?
- a unit test case in FilterPushdownSuite
- an end-to-end test in limit.sql

Author: Reynold Xin <rxin@databricks.com>

Closes #14713 from rxin/SPARK-16994.
2016-08-19 21:11:35 +08:00
Jeff Zhang 072acf5e14 [SPARK-16965][MLLIB][PYSPARK] Fix bound checking for SparseVector.
## What changes were proposed in this pull request?

1. In scala, add negative low bound checking and put all the low/upper bound checking in one place
2. In python, add low/upper bound checking of indices.

## How was this patch tested?

unit test added

Author: Jeff Zhang <zjffdu@apache.org>

Closes #14555 from zjffdu/SPARK-16965.
2016-08-19 12:38:15 +01:00
Yanbo Liang 864be9359a [SPARK-17141][ML] MinMaxScaler should remain NaN value.
## What changes were proposed in this pull request?
In the existing code, ```MinMaxScaler``` handle ```NaN``` value indeterminately.
* If a column has identity value, that is ```max == min```, ```MinMaxScalerModel``` transformation will output ```0.5``` for all rows even the original value is ```NaN```.
* Otherwise, it will remain ```NaN``` after transformation.

I think we should unify the behavior by remaining ```NaN``` value at any condition, since we don't know how to transform a ```NaN``` value. In Python sklearn, it will throw exception when there is ```NaN``` in the dataset.

## How was this patch tested?
Unit tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #14716 from yanboliang/spark-17141.
2016-08-19 03:23:16 -07:00
Nick Lavers 5377fc6236 [SPARK-16961][CORE] Fixed off-by-one error that biased randomizeInPlace
JIRA issue link:
https://issues.apache.org/jira/browse/SPARK-16961

Changed one line of Utils.randomizeInPlace to allow elements to stay in place.

Created a unit test that runs a Pearson's chi squared test to determine whether the output diverges significantly from a uniform distribution.

Author: Nick Lavers <nick.lavers@videoamp.com>

Closes #14551 from nicklavers/SPARK-16961-randomizeInPlace.
2016-08-19 10:11:59 +01:00
sethah 287bea1305 [SPARK-7159][ML] Add multiclass logistic regression to Spark ML
## What changes were proposed in this pull request?

This patch adds a new estimator/transformer `MultinomialLogisticRegression` to spark ML.

JIRA: [SPARK-7159](https://issues.apache.org/jira/browse/SPARK-7159)

## How was this patch tested?

Added new test suite `MultinomialLogisticRegressionSuite`.

## Approach

### Do not use a "pivot" class in the algorithm formulation

Many implementations of multinomial logistic regression treat the problem as K - 1 independent binary logistic regression models where K is the number of possible outcomes in the output variable. In this case, one outcome is chosen as a "pivot" and the other K - 1 outcomes are regressed against the pivot. This is somewhat undesirable since the coefficients returned will be different for different choices of pivot variables. An alternative approach to the problem models class conditional probabilites using the softmax function and will return uniquely identifiable coefficients (assuming regularization is applied). This second approach is used in R's glmnet and was also recommended by dbtsai.

### Separate multinomial logistic regression and binary logistic regression

The initial design makes multinomial logistic regression a separate estimator/transformer than the existing LogisticRegression estimator/transformer. An alternative design would be to merge them into one.

**Arguments for:**

* The multinomial case without pivot is distinctly different than the current binary case since the binary case uses a pivot class.
* The current logistic regression model in ML uses a vector of coefficients and a scalar intercept. In the multinomial case, we require a matrix of coefficients and a vector of intercepts. There are potential workarounds for this issue if we were to merge the two estimators, but none are particularly elegant.

**Arguments against:**

* It may be inconvenient for users to have to switch the estimator class when transitioning between binary and multiclass (although the new multinomial estimator can be used for two class outcomes).
* Some portions of the code are repeated.

This is a major design point and warrants more discussion.

### Mean centering

When no regularization is applied, the coefficients will not be uniquely identifiable. This is not hard to show and is discussed in further detail [here](https://core.ac.uk/download/files/153/6287975.pdf). R's glmnet deals with this by choosing the minimum l2 regularized solution (i.e. mean centering). Additionally, the intercepts are never regularized so they are always mean centered. This is the approach taken in this PR as well.

### Feature scaling

In current ML logistic regression, the features are always standardized when running the optimization algorithm. They are always returned to the user in the original feature space, however. This same approach is maintained in this patch as well, but the implementation details are different. In ML logistic regression, the unregularized feature values are divided by the column standard deviation in every gradient update iteration. In contrast, MLlib transforms the entire input dataset to the scaled space _before_ optimizaton. In ML, this means that `numFeatures * numClasses` extra scalar divisions are required in every iteration. Performance testing shows that this has significant (4x in some cases) slow downs in each iteration. This can be avoided by transforming the input to the scaled space ala MLlib once, before iteration begins. This does add some overhead initially, but can make significant time savings in some cases.

One issue with this approach is that if the input data is already cached, there may not be enough memory to cache the transformed data, which would make the algorithm _much_ slower. The tradeoffs here merit more discussion.

### Specifying and inferring the number of outcome classes

The estimator checks the dataframe label column for metadata which specifies the number of values. If they are not specified, the length of the `histogram` variable is used, which is essentially the maximum value found in the column. The assumption then, is that the labels are zero-indexed when they are provided to the algorithm.

## Performance

Below are some performance tests I have run so far. I am happy to add more cases or trials if we deem them necessary.

Test cluster: 4 bare metal nodes, 128 GB RAM each, 48 cores each

Notes:

* Time in units of seconds
* Metric is classification accuracy

| algo   |   elasticNetParam | fitIntercept   |   metric |   maxIter |   numPoints |   numClasses |   numFeatures |    time | standardization   |   regParam |
|--------|-------------------|----------------|----------|-----------|-------------|--------------|---------------|---------|-------------------|------------|
| ml     |                 0 | true           | 0.746415 |        30 |      100000 |            3 |        100000 | 327.923 | true              |          0 |
| mllib  |                 0 | true           | 0.743785 |        30 |      100000 |            3 |        100000 | 390.217 | true              |          0 |

| algo   |   elasticNetParam | fitIntercept   |   metric |   maxIter |   numPoints |   numClasses |   numFeatures |    time | standardization   |   regParam |
|--------|-------------------|----------------|----------|-----------|-------------|--------------|---------------|---------|-------------------|------------|
| ml     |                 0 | true           | 0.973238 |        30 |     2000000 |            3 |         10000 | 385.476 | true              |          0 |
| mllib  |                 0 | true           | 0.949828 |        30 |     2000000 |            3 |         10000 | 550.403 | true              |          0 |

| algo   |   elasticNetParam | fitIntercept   |   metric |   maxIter |   numPoints |   numClasses |   numFeatures |    time | standardization   |   regParam |
|--------|-------------------|----------------|----------|-----------|-------------|--------------|---------------|---------|-------------------|------------|
| mllib  |                 0 | true           | 0.864358 |        30 |     2000000 |            3 |         10000 | 543.359 | true              |        0.1 |
| ml     |                 0 | true           | 0.867418 |        30 |     2000000 |            3 |         10000 | 401.955 | true              |        0.1 |

| algo   |   elasticNetParam | fitIntercept   |   metric |   maxIter |   numPoints |   numClasses |   numFeatures |    time | standardization   |   regParam |
|--------|-------------------|----------------|----------|-----------|-------------|--------------|---------------|---------|-------------------|------------|
| ml     |                 1 | true           | 0.807449 |        30 |     2000000 |            3 |         10000 | 334.892 | true              |       0.05 |

| algo   |   elasticNetParam | fitIntercept   |   metric |   maxIter |   numPoints |   numClasses |   numFeatures |    time | standardization   |   regParam |
|--------|-------------------|----------------|----------|-----------|-------------|--------------|---------------|---------|-------------------|------------|
| ml     |                 0 | true           | 0.602006 |        30 |     2000000 |          500 |           100 | 112.319 | true              |          0 |
| mllib  |                 0 | true           | 0.567226 |        30 |     2000000 |          500 |           100 | 263.768 | true              |          0 |e           | 0.567226 |        30 |     2000000 |          500 |           100 | 263.768 | true              |          0 |

## References

Friedman, et al. ["Regularization Paths for Generalized Linear Models via Coordinate Descent"](https://core.ac.uk/download/files/153/6287975.pdf)
[http://web.stanford.edu/~hastie/glmnet/glmnet_alpha.html](http://web.stanford.edu/~hastie/glmnet/glmnet_alpha.html)

## Follow up items
* Consider using level 2 BLAS routines in the gradient computations - [SPARK-17134](https://issues.apache.org/jira/browse/SPARK-17134)
* Add model summary for MLOR - [SPARK-17139](https://issues.apache.org/jira/browse/SPARK-17139)
* Add initial model to MLOR and add test for intercept priors - [SPARK-17140](https://issues.apache.org/jira/browse/SPARK-17140)
* Python API - [SPARK-17138](https://issues.apache.org/jira/browse/SPARK-17138)
* Consider changing the tree aggregation level for MLOR/BLOR or making it user configurable to avoid memory problems with high dimensional data - [SPARK-17090](https://issues.apache.org/jira/browse/SPARK-17090)
* Refactor helper classes out of `LogisticRegression.scala` - [SPARK-17135](https://issues.apache.org/jira/browse/SPARK-17135)
* Design optimizer interface for added flexibility in ML algos - [SPARK-17136](https://issues.apache.org/jira/browse/SPARK-17136)
* Support compressing the coefficients and intercepts for MLOR models - [SPARK-17137](https://issues.apache.org/jira/browse/SPARK-17137)

Author: sethah <seth.hendrickson16@gmail.com>

Closes #13796 from sethah/SPARK-7159_M.
2016-08-18 22:16:48 -07:00
Reynold Xin b482c09fa2 HOTFIX: compilation broken due to protected ctor. 2016-08-18 19:02:32 -07:00
petermaxlee f5472dda51 [SPARK-16947][SQL] Support type coercion and foldable expression for inline tables
## What changes were proposed in this pull request?
This patch improves inline table support with the following:

1. Support type coercion.
2. Support using foldable expressions. Previously only literals were supported.
3. Improve error message handling.
4. Improve test coverage.

## How was this patch tested?
Added a new unit test suite ResolveInlineTablesSuite and a new file-based end-to-end test inline-table.sql.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14676 from petermaxlee/SPARK-16947.
2016-08-19 09:19:47 +08:00
Xusen Yin b72bb62d42 [SPARK-16447][ML][SPARKR] LDA wrapper in SparkR
## What changes were proposed in this pull request?

Add LDA Wrapper in SparkR with the following interfaces:

- spark.lda(data, ...)

- spark.posterior(object, newData, ...)

- spark.perplexity(object, ...)

- summary(object)

- write.ml(object)

- read.ml(path)

## How was this patch tested?

Test with SparkR unit test.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #14229 from yinxusen/SPARK-16447.
2016-08-18 05:33:52 -07:00
petermaxlee 68f5087d21 [SPARK-17117][SQL] 1 / NULL should not fail analysis
## What changes were proposed in this pull request?
This patch fixes the problem described in SPARK-17117, i.e. "SELECT 1 / NULL" throws an analysis exception:

```
org.apache.spark.sql.AnalysisException: cannot resolve '(1 / NULL)' due to data type mismatch: differing types in '(1 / NULL)' (int and null).
```

The problem is that division type coercion did not take null type into account.

## How was this patch tested?
A unit test for the type coercion, and a few end-to-end test cases using SQLQueryTestSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14695 from petermaxlee/SPARK-17117.
2016-08-18 13:44:13 +02:00
Eric Liang 412dba63b5 [SPARK-17069] Expose spark.range() as table-valued function in SQL
## What changes were proposed in this pull request?

This adds analyzer rules for resolving table-valued functions, and adds one builtin implementation for range(). The arguments for range() are the same as those of `spark.range()`.

## How was this patch tested?

Unit tests.

cc hvanhovell

Author: Eric Liang <ekl@databricks.com>

Closes #14656 from ericl/sc-4309.
2016-08-18 13:33:55 +02:00
Stavros Kontopoulos b81421afb0 [SPARK-17087][MESOS] Documentation for Making Spark on Mesos honor port restrictions
## What changes were proposed in this pull request?

- adds documentation for https://issues.apache.org/jira/browse/SPARK-11714

## How was this patch tested?
Doc no test needed.

Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>

Closes #14667 from skonto/add_doc.
2016-08-18 12:19:19 +01:00
Liang-Chi Hsieh e82dbe600e [SPARK-17107][SQL] Remove redundant pushdown rule for Union
## What changes were proposed in this pull request?

The `Optimizer` rules `PushThroughSetOperations` and `PushDownPredicate` have a redundant rule to push down `Filter` through `Union`. We should remove it.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14687 from viirya/remove-extra-pushdown.
2016-08-18 12:45:56 +02:00
Reynold Xin 1748f82410 [SPARK-16391][SQL] Support partial aggregation for reduceGroups
## What changes were proposed in this pull request?
This patch introduces a new private ReduceAggregator interface that is a subclass of Aggregator. ReduceAggregator only requires a single associative and commutative reduce function. ReduceAggregator is also used to implement KeyValueGroupedDataset.reduceGroups in order to support partial aggregation.

Note that the pull request was initially done by viirya.

## How was this patch tested?
Covered by original tests for reduceGroups, as well as a new test suite for ReduceAggregator.

Author: Reynold Xin <rxin@databricks.com>
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14576 from rxin/reduceAggregator.
2016-08-18 16:37:25 +08:00
petermaxlee 3e6ef2e8a4 [SPARK-17034][SQL] Minor code cleanup for UnresolvedOrdinal
## What changes were proposed in this pull request?
I was looking at the code for UnresolvedOrdinal and made a few small changes to make it slightly more clear:

1. Rename the rule to SubstituteUnresolvedOrdinals which is more consistent with other rules that start with verbs. Note that this is still inconsistent with CTESubstitution and WindowsSubstitution.
2. Broke the test suite down from a single test case to three test cases.

## How was this patch tested?
This is a minor cleanup.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14672 from petermaxlee/SPARK-17034.
2016-08-18 16:17:01 +08:00
Liang-Chi Hsieh 10204b9d29 [SPARK-16995][SQL] TreeNodeException when flat mapping RelationalGroupedDataset created from DataFrame containing a column created with lit/expr
## What changes were proposed in this pull request?

A TreeNodeException is thrown when executing the following minimal example in Spark 2.0.

    import spark.implicits._
    case class test (x: Int, q: Int)

    val d = Seq(1).toDF("x")
    d.withColumn("q", lit(0)).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show
    d.withColumn("q", expr("0")).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show

The problem is at `FoldablePropagation`. The rule will do `transformExpressions` on `LogicalPlan`. The query above contains a `MapGroups` which has a parameter `dataAttributes:Seq[Attribute]`. One attributes in `dataAttributes` will be transformed to an `Alias(literal(0), _)` in `FoldablePropagation`. `Alias` is not an `Attribute` and causes the error.

We can't easily detect such type inconsistency during transforming expressions. A direct approach to this problem is to skip doing `FoldablePropagation` on object operators as they should not contain such expressions.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #14648 from viirya/flat-mapping.
2016-08-18 13:24:12 +08:00
Xin Ren e6bef7d52f [SPARK-17038][STREAMING] fix metrics retrieval source of 'lastReceivedBatch'
https://issues.apache.org/jira/browse/SPARK-17038

## What changes were proposed in this pull request?

StreamingSource's lastReceivedBatch_submissionTime, lastReceivedBatch_processingTimeStart, and lastReceivedBatch_processingTimeEnd all use data from lastCompletedBatch instead of lastReceivedBatch.

In particular, this makes it impossible to match lastReceivedBatch_records with a batchID/submission time.

This is apparent when looking at StreamingSource.scala, lines 89-94.

## How was this patch tested?

Manually running unit tests on local laptop

Author: Xin Ren <iamshrek@126.com>

Closes #14681 from keypointt/SPARK-17038.
2016-08-17 16:31:42 -07:00
Tathagata Das d60af8f6aa [SPARK-17096][SQL][STREAMING] Improve exception string reported through the StreamingQueryListener
## What changes were proposed in this pull request?

Currently, the stackTrace (as `Array[StackTraceElements]`) reported through StreamingQueryListener.onQueryTerminated is useless as it has the stack trace of where StreamingQueryException is defined, not the stack trace of underlying exception.  For example, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.stackTrace` will have
```
org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:211)
org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124)
```
This is basically useless, as it is location where the StreamingQueryException was defined. What we want is

Here is the right way to reason about what should be posted as through StreamingQueryListener.onQueryTerminated
- The actual exception could either be a SparkException, or an arbitrary exception.
  - SparkException reports the relevant executor stack trace of a failed task as a string in the the exception message. The `Array[StackTraceElements]` returned by `SparkException.stackTrace()` is mostly irrelevant.
  - For any arbitrary exception, the `Array[StackTraceElements]` returned by `exception.stackTrace()` may be relevant.
- When there is an error in a streaming query, it's hard to reason whether the `Array[StackTraceElements]` is useful or not. In fact, it is not clear whether it is even useful to report the stack trace as this array of Java objects. It may be sufficient to report the strack trace as a string, along with the message. This is how Spark reported executor stra
- Hence, this PR simplifies the API by removing the array `stackTrace` from `QueryTerminated`. Instead the `exception` returns a string containing the message and the stack trace of the actual underlying exception that failed the streaming query (i.e. not that of the StreamingQueryException). If anyone is interested in the actual stack trace as an array, can always access them through `streamingQuery.exception` which returns the exception object.

With this change, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.exception` will be
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent failure: Lost task 1.0 in stage 0.0 (TID 1, localhost): java.lang.ArithmeticException: / by zero
	at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply$mcII$sp(StreamingQueryListenerSuite.scala:153)
	at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153)
	at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:232)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:226)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
	at org.apache.spark.scheduler.Task.run(Task.scala:86)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:744)

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1429)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1417)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1416)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1416)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
...
```
It contains the relevant executor stack trace. In a case non-SparkException, if the streaming source MemoryStream throws an exception, exception message will have the relevant stack trace.
```
java.lang.RuntimeException: this is the exception message
	at org.apache.spark.sql.execution.streaming.MemoryStream.getBatch(memory.scala:103)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:316)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:313)
	at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
	at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
	at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25)
	at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
	at org.apache.spark.sql.execution.streaming.StreamProgress.flatMap(StreamProgress.scala:25)
	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:313)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:197)
	at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43)
	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:187)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124)
```

Note that this change in the public `QueryTerminated` class is okay as the APIs are still experimental.

## How was this patch tested?
Unit tests that test whether the right information is present in the exception message reported through QueryTerminated object.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #14675 from tdas/SPARK-17096.
2016-08-17 13:31:34 -07:00
Steve Loughran cc97ea188e [SPARK-16736][CORE][SQL] purge superfluous fs calls
A review of the code, working back from Hadoop's `FileSystem.exists()` and `FileSystem.isDirectory()` code, then removing uses of the calls when superfluous.

1. delete is harmless if called on a nonexistent path, so don't do any checks before deletes
1. any `FileSystem.exists()`  check before `getFileStatus()` or `open()` is superfluous as the operation itself does the check. Instead the `FileNotFoundException` is caught and triggers the downgraded path. When a `FileNotFoundException` was thrown before, the code still creates a new FNFE with the error messages. Though now the inner exceptions are nested, for easier diagnostics.

Initially, relying on Jenkins test runs.

One troublespot here is that some of the codepaths are clearly error situations; it's not clear that they have coverage anyway. Trying to create the failure conditions in tests would be ideal, but it will also be hard.

Author: Steve Loughran <stevel@apache.org>

Closes #14371 from steveloughran/cloud/SPARK-16736-superfluous-fs-calls.
2016-08-17 11:43:01 -07:00
Yanbo Liang 4d92af310a [SPARK-16446][SPARKR][ML] Gaussian Mixture Model wrapper in SparkR
## What changes were proposed in this pull request?
Gaussian Mixture Model wrapper in SparkR, similarly to R's ```mvnormalmixEM```.

## How was this patch tested?
Unit test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #14392 from yanboliang/spark-16446.
2016-08-17 11:18:33 -07:00
Marcelo Vanzin e3fec51fa1 [SPARK-16930][YARN] Fix a couple of races in cluster app initialization.
There are two narrow races that could cause the ApplicationMaster to miss
when the user application instantiates the SparkContext, which could cause
app failures when nothing was wrong with the app. It was also possible for
a failing application to get stuck in the loop that waits for the context
for a long time, instead of failing quickly.

The change uses a promise to track the SparkContext instance, which gets
rid of the races and allows for some simplification of the code.

Tested with existing unit tests, and a new one being added to test the
timeout code.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14542 from vanzin/SPARK-16930.
2016-08-17 11:12:21 -07:00
Wenchen Fan 928ca1c6d1 [SPARK-17102][SQL] bypass UserDefinedGenerator for json format check
## What changes were proposed in this pull request?

We use reflection to convert `TreeNode` to json string, and currently don't support arbitrary object. `UserDefinedGenerator` takes a function object, so we should skip json format test for it, or the tests can be flacky, e.g. `DataFrameSuite.simple explode`, this test always fail with scala 2.10(branch 1.6 builds with scala 2.10 by default), but pass with scala 2.11(master branch builds with scala 2.11 by default).

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14679 from cloud-fan/json.
2016-08-17 09:31:22 -07:00
Herman van Hovell 0b0c8b95e3 [SPARK-17106] [SQL] Simplify the SubqueryExpression interface
## What changes were proposed in this pull request?
The current subquery expression interface contains a little bit of technical debt in the form of a few different access paths to get and set the query contained by the expression. This is confusing to anyone who goes over this code.

This PR unifies these access paths.

## How was this patch tested?
(Existing tests)

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14685 from hvanhovell/SPARK-17106.
2016-08-17 07:03:24 -07:00
Kazuaki Ishizaki 56d86742d2 [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB
## What changes were proposed in this pull request?

This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method.

Here is [the original PR](https://github.com/apache/spark/pull/13243) for SPARK-15285. However, it breaks a build with Scala 2.10 since Scala 2.10 does not a case class with large number of members. Thus, it was reverted by [this commit](fa244e5a90).

## How was this patch tested?

Added new tests by using `DefinedByConstructorParams` instead of case class for scala-2.10

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #14670 from kiszk/SPARK-15285-2.
2016-08-17 21:34:57 +08:00
wm624@hotmail.com 363793f2bf [SPARK-16444][SPARKR] Isotonic Regression wrapper in SparkR
## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

Add Isotonic Regression wrapper in SparkR

Wrappers in R and Scala are added.
Unit tests
Documentation

## How was this patch tested?
Manually tested with sudo ./R/run-tests.sh

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #14182 from wangmiao1981/isoR.
2016-08-17 06:15:04 -07:00
jiangxingbo 4d0cc84afc [SPARK-17032][SQL] Add test cases for methods in ParserUtils.
## What changes were proposed in this pull request?

Currently methods in `ParserUtils` are tested indirectly, we should add test cases in `ParserUtilsSuite` to verify their integrity directly.

## How was this patch tested?

New test cases in `ParserUtilsSuite`

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #14620 from jiangxb1987/parserUtils.
2016-08-17 14:22:36 +02:00
mvervuurt 0f6aa8afaa [MINOR][DOC] Fix the descriptions for properties argument in the documenation for jdbc APIs
## What changes were proposed in this pull request?

This should be credited to mvervuurt. The main purpose of this PR is
 - simply to include the change for the same instance in `DataFrameReader` just to match up.
 - just avoid duplicately verifying the PR (as I already did).

The documentation for both should be the same because both assume the `properties` should be  the same `dict` for the same option.

## How was this patch tested?

Manually building Python documentation.

This will produce the output as below:

- `DataFrameReader`

![2016-08-17 11 12 00](https://cloud.githubusercontent.com/assets/6477701/17722764/b3f6568e-646f-11e6-8b75-4fb672f3f366.png)

- `DataFrameWriter`

![2016-08-17 11 12 10](https://cloud.githubusercontent.com/assets/6477701/17722765/b58cb308-646f-11e6-841a-32f19800d139.png)

Closes #14624

Author: hyukjinkwon <gurwls223@gmail.com>
Author: mvervuurt <m.a.vervuurt@gmail.com>

Closes #14677 from HyukjinKwon/typo-python.
2016-08-16 23:12:59 -07:00
Herman van Hovell f7c9ff57c1 [SPARK-17068][SQL] Make view-usage visible during analysis
## What changes were proposed in this pull request?
This PR adds a field to subquery alias in order to make the usage of views in a resolved `LogicalPlan` more visible (and more understandable).

For example, the following view and query:
```sql
create view constants as select 1 as id union all select 1 union all select 42
select * from constants;
```
...now yields the following analyzed plan:
```
Project [id#39]
+- SubqueryAlias c, `default`.`constants`
   +- Project [gen_attr_0#36 AS id#39]
      +- SubqueryAlias gen_subquery_0
         +- Union
            :- Union
            :  :- Project [1 AS gen_attr_0#36]
            :  :  +- OneRowRelation$
            :  +- Project [1 AS gen_attr_1#37]
            :     +- OneRowRelation$
            +- Project [42 AS gen_attr_2#38]
               +- OneRowRelation$
```
## How was this patch tested?
Added tests for the two code paths in `SessionCatalogSuite` (sql/core) and `HiveMetastoreCatalogSuite` (sql/hive)

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14657 from hvanhovell/SPARK-17068.
2016-08-16 23:09:53 -07:00
Herman van Hovell 4a2c375be2 [SPARK-17084][SQL] Rename ParserUtils.assert to validate
## What changes were proposed in this pull request?
This PR renames `ParserUtils.assert` to `ParserUtils.validate`. This is done because this method is used to check requirements, and not to check if the program is in an invalid state.

## How was this patch tested?
Simple rename. Compilation should do.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14665 from hvanhovell/SPARK-17084.
2016-08-16 21:35:39 -07:00
sandy e28a8c5899 [SPARK-17089][DOCS] Remove api doc link for mapReduceTriplets operator
## What changes were proposed in this pull request?

Remove the api doc link for mapReduceTriplets operator because in latest api they are remove so when user link to that api they will not get mapReduceTriplets there so its more good to remove than confuse the user.

## How was this patch tested?
Run all the test cases

![screenshot from 2016-08-16 23-08-25](https://cloud.githubusercontent.com/assets/8075390/17709393/8cfbf75a-6406-11e6-98e6-38f7b319d833.png)

Author: sandy <phalodi@gmail.com>

Closes #14669 from phalodi/SPARK-17089.
2016-08-16 12:50:55 -07:00
Felix Cheung c34b546d67 [SPARK-16519][SPARKR] Handle SparkR RDD generics that create warnings in R CMD check
## What changes were proposed in this pull request?

Rename RDD functions for now to avoid CRAN check warnings.
Some RDD functions are sharing generics with DataFrame functions (hence the problem) so after the renames we need to add new generics, for now.

## How was this patch tested?

unit tests

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #14626 from felixcheung/rrddfunctions.
2016-08-16 11:19:18 -07:00
Yanbo Liang d37ea3c09c [MINOR][SPARKR] spark.glm weightCol should in the signature.
## What changes were proposed in this pull request?
Fix the issue that ```spark.glm``` ```weightCol``` should in the signature.

## How was this patch tested?
Existing tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #14641 from yanboliang/weightCol.
2016-08-16 10:52:35 -07:00
Dongjoon Hyun 12a89e55cb [SPARK-17035] [SQL] [PYSPARK] Improve Timestamp not to lose precision for all cases
## What changes were proposed in this pull request?

`PySpark` loses `microsecond` precision for some corner cases during converting `Timestamp` into `Long`. For example, for the following `datetime.max` value should be converted a value whose last 6 digits are '999999'. This PR improves the logic not to lose precision for all cases.

**Corner case**
```python
>>> datetime.datetime.max
datetime.datetime(9999, 12, 31, 23, 59, 59, 999999)
```

**Before**
```python
>>> from datetime import datetime
>>> from pyspark.sql import Row
>>> from pyspark.sql.types import StructType, StructField, TimestampType
>>> schema = StructType([StructField("dt", TimestampType(), False)])
>>> [schema.toInternal(row) for row in [{"dt": datetime.max}]]
[(253402329600000000,)]
```

**After**
```python
>>> [schema.toInternal(row) for row in [{"dt": datetime.max}]]
[(253402329599999999,)]
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14631 from dongjoon-hyun/SPARK-17035.
2016-08-16 10:01:30 -07:00
linbojin 6f0988b129 [MINOR][DOC] Correct code snippet results in quick start documentation
## What changes were proposed in this pull request?

As README.md file is updated over time. Some code snippet outputs are not correct based on new README.md file. For example:
```
scala> textFile.count()
res0: Long = 126
```
should be
```
scala> textFile.count()
res0: Long = 99
```
This pr is to add comments to point out this problem so that new spark learners have a correct reference.
Also, fixed a samll bug, inside current documentation, the outputs of linesWithSpark.count() without and with cache are different (one is 15 and the other is 19)
```
scala> val linesWithSpark = textFile.filter(line => line.contains("Spark"))
linesWithSpark: org.apache.spark.rdd.RDD[String] = MapPartitionsRDD[2] at filter at <console>:27

scala> textFile.filter(line => line.contains("Spark")).count() // How many lines contain "Spark"?
res3: Long = 15

...

scala> linesWithSpark.cache()
res7: linesWithSpark.type = MapPartitionsRDD[2] at filter at <console>:27

scala> linesWithSpark.count()
res8: Long = 19
```

## How was this patch tested?

manual test:  run `$ SKIP_API=1 jekyll serve --watch`

Author: linbojin <linbojin203@gmail.com>

Closes #14645 from linbojin/quick-start-documentation.
2016-08-16 11:37:54 +01:00
Herman van Hovell 8fdc6ce400 [SPARK-16964][SQL] Remove private[hive] from sql.hive.execution package
## What changes were proposed in this pull request?
This PR is a small follow-up to https://github.com/apache/spark/pull/14554. This also widens the visibility of a few (similar) Hive classes.

## How was this patch tested?
No test. Only a visibility change.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14654 from hvanhovell/SPARK-16964-hive.
2016-08-16 01:12:27 -07:00
Sean Zhong 7b65030e7a [SPARK-17034][SQL] adds expression UnresolvedOrdinal to represent the ordinals in GROUP BY or ORDER BY
## What changes were proposed in this pull request?

This PR adds expression `UnresolvedOrdinal` to represent the ordinal in GROUP BY or ORDER BY, and fixes the rules when resolving ordinals.

Ordinals in GROUP BY or ORDER BY like `1` in `order by 1` or `group by 1` should be considered as unresolved before analysis. But in current code, it uses `Literal` expression to store the ordinal. This is inappropriate as `Literal` itself is a resolved expression, it gives the user a wrong message that the ordinals has already been resolved.

### Before this change

Ordinal is stored as `Literal` expression

```
scala> sc.setLogLevel("TRACE")
scala> sql("select a from t group by 1 order by 1")
...
'Sort [1 ASC], true
 +- 'Aggregate [1], ['a]
     +- 'UnresolvedRelation `t
```

For query:

```
scala> Seq(1).toDF("a").createOrReplaceTempView("t")
scala> sql("select count(a), a from t group by 2 having a > 0").show
```

During analysis, the intermediate plan before applying rule `ResolveAggregateFunctions` is:

```
'Filter ('a > 0)
   +- Aggregate [2], [count(1) AS count(1)#83L, a#81]
        +- LocalRelation [value#7 AS a#9]
```

Before this PR, rule `ResolveAggregateFunctions` believes all expressions of `Aggregate` have already been resolved, and tries to resolve the expressions in `Filter` directly. But this is wrong, as ordinal `2` in Aggregate is not really resolved!

### After this change

Ordinals are stored as `UnresolvedOrdinal`.

```
scala> sc.setLogLevel("TRACE")
scala> sql("select a from t group by 1 order by 1")
...
'Sort [unresolvedordinal(1) ASC], true
 +- 'Aggregate [unresolvedordinal(1)], ['a]
      +- 'UnresolvedRelation `t`
```

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14616 from clockfly/spark-16955.
2016-08-16 15:51:30 +08:00
Wenchen Fan 7de30d6e9e [SPARK-16916][SQL] serde/storage properties should not have limitations
## What changes were proposed in this pull request?

`CatalogStorageFormat.properties` can be used in 2 ways:

1. for hive tables, it stores the serde properties.
2. for data source tables, it stores the data source options, e.g. `path`, `skipHiveMetadata`, etc.

however, both of them have nothing to do with data source properties, e.g. `spark.sql.sources.provider`, so they should not have limitations about data source properties.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14506 from cloud-fan/table-prop.
2016-08-15 21:43:41 -07:00
Shixiong Zhu 268b71d0d7 [SPARK-17065][SQL] Improve the error message when encountering an incompatible DataSourceRegister
## What changes were proposed in this pull request?

Add an instruction to ask the user to remove or upgrade the incompatible DataSourceRegister in the error message.

## How was this patch tested?

Test command:
```
build/sbt -Dscala-2.10 package
SPARK_SCALA_VERSION=2.10 bin/spark-shell --packages ai.h2o:sparkling-water-core_2.10:1.6.5

scala> Seq(1).toDS().write.format("parquet").save("foo")
```

Before:
```
java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated
	at java.util.ServiceLoader.fail(ServiceLoader.java:232)
	at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
	at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
	at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
...
Caused by: java.lang.NoClassDefFoundError: org/apache/spark/Logging
	at java.lang.ClassLoader.defineClass1(Native Method)
	at java.lang.ClassLoader.defineClass(ClassLoader.java:760)
	at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
	at java.net.URLClassLoader.defineClass(URLClassLoader.java:467)
	at java.net.URLClassLoader.access$100(URLClassLoader.java:73)
	at java.net.URLClassLoader$1.run(URLClassLoader.java:368)
	at java.net.URLClassLoader$1.run(URLClassLoader.java:362)
	at java.security.AccessController.doPrivileged(Native Method)
...
```

After:

```
java.lang.ClassNotFoundException: Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated
	at org.apache.spark.sql.execution.datasources.DataSource.lookupDataSource(DataSource.scala:178)
	at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:79)
	at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:79)
	at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:441)
	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:213)
	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:196)
...
```

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #14651 from zsxwing/SPARK-17065.
2016-08-15 15:55:32 -07:00
Davies Liu fffb0c0d19 [SPARK-16700][PYSPARK][SQL] create DataFrame from dict/Row with schema
## What changes were proposed in this pull request?

In 2.0, we verify the data type against schema for every row for safety, but with performance cost, this PR make it optional.

When we verify the data type for StructType, it does not support all the types we support in infer schema (for example, dict), this PR fix that to make them consistent.

For Row object which is created using named arguments, the order of fields are sorted by name, they may be not different than the order in provided schema, this PR fix that by ignore the order of fields in this case.

## How was this patch tested?

Created regression tests for them.

Author: Davies Liu <davies@databricks.com>

Closes #14469 from davies/py_dict.
2016-08-15 12:41:27 -07:00
Marcelo Vanzin 5da6c4b24f [SPARK-16671][CORE][SQL] Consolidate code to do variable substitution.
Both core and sql have slightly different code that does variable substitution
of config values. This change refactors that code and encapsulates the logic
of reading config values and expading variables in a new helper class, which
can be configured so that both core and sql can use it without losing existing
functionality, and allows for easier testing and makes it easier to add more
features in the future.

Tested with existing and new unit tests, and by running spark-shell with
some configs referencing variables and making sure it behaved as expected.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14468 from vanzin/SPARK-16671.
2016-08-15 11:09:54 -07:00
Junyang Qian 564fe614c1 [SPARK-16508][SPARKR] Split docs for arrange and orderBy methods
## What changes were proposed in this pull request?

This PR splits arrange and orderBy methods according to their functionality (the former for sorting sparkDataFrame and the latter for windowSpec).

## How was this patch tested?

![screen shot 2016-08-06 at 6 39 19 pm](https://cloud.githubusercontent.com/assets/15318264/17459969/51eade28-5c05-11e6-8ca1-8d8a8e344bab.png)
![screen shot 2016-08-06 at 6 39 29 pm](https://cloud.githubusercontent.com/assets/15318264/17459966/51e3c246-5c05-11e6-8d35-3e905ca48676.png)
![screen shot 2016-08-06 at 6 40 02 pm](https://cloud.githubusercontent.com/assets/15318264/17459967/51e650ec-5c05-11e6-8698-0f037f5199ff.png)

Author: Junyang Qian <junyangq@databricks.com>

Closes #14522 from junyangq/SPARK-16508-0.
2016-08-15 11:03:03 -07:00
WeichenXu 3d8bfe7a39 [SPARK-16934][ML][MLLIB] Update LogisticCostAggregator serialization code to make it consistent with LinearRegression
## What changes were proposed in this pull request?

Update LogisticCostAggregator serialization code to make it consistent with #14109

## How was this patch tested?
MLlib 2.0:
![image](https://cloud.githubusercontent.com/assets/19235986/17649601/5e2a79ac-61ee-11e6-833c-3bd8b5250470.png)

After this PR:
![image](https://cloud.githubusercontent.com/assets/19235986/17649599/52b002ae-61ee-11e6-9402-9feb3439880f.png)

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #14520 from WeichenXu123/improve_logistic_regression_costfun.
2016-08-15 06:38:30 -07:00
Yanbo Liang ddf0d1e3fe [TRIVIAL][ML] Fix LogisticRegression typo in error message.
## What changes were proposed in this pull request?
Fix ```LogisticRegression``` typo in error message.

## How was this patch tested?
Docs change, no new tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #14633 from yanboliang/lr-typo.
2016-08-15 10:11:29 +01:00
Stavros Kontopoulos 1a028bdefa [SPARK-11714][MESOS] Make Spark on Mesos honor port restrictions on coarse grain mode
- Make mesos coarse grained scheduler accept port offers and pre-assign ports

Previous attempt was for fine grained: https://github.com/apache/spark/pull/10808

Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Author: Stavros Kontopoulos <stavros.kontopoulos@typesafe.com>

Closes #11157 from skonto/honour_ports_coarse.
2016-08-15 09:55:32 +01:00
Zhenglai Zhang 2a3d286f34 [WIP][MINOR][TYPO] Fix several trivival typos
## What changes were proposed in this pull request?

* Fixed one typo `"overriden"` as `"overridden"`, also make sure no other same typo.
* Fixed one typo `"lowcase"` as `"lowercase"`, also make sure no other same typo.

## How was this patch tested?

Since the change is very tiny, so I just make sure compilation is successful.
I am new to the spark community,  please feel free to let me do other necessary steps.

Thanks in advance!

----
Updated: Found another typo `lowcase` later and fixed then in the same patch

Author: Zhenglai Zhang <zhenglaizhang@hotmail.com>

Closes #14622 from zhenglaizhang/fixtypo.
2016-08-14 16:10:34 +01:00
zero323 0ebf7c1bff [SPARK-17027][ML] Avoid integer overflow in PolynomialExpansion.getPolySize
## What changes were proposed in this pull request?

Replaces custom choose function with o.a.commons.math3.CombinatoricsUtils.binomialCoefficient

## How was this patch tested?

Spark unit tests

Author: zero323 <zero323@users.noreply.github.com>

Closes #14614 from zero323/SPARK-17027.
2016-08-14 11:59:24 +01:00