Commit graph

17707 commits

Author SHA1 Message Date
Ding Fei 6ee28423ad Fix two comments since Actor is not used anymore.
## What changes were proposed in this pull request?

Fix two comments since Actor is not used anymore.

Author: Ding Fei <danis@danix>

Closes #15251 from danix800/comment-fixing.
2016-09-26 23:09:51 -07:00
Yanbo Liang 93c743f1ac [SPARK-17577][FOLLOW-UP][SPARKR] SparkR spark.addFile supports adding directory recursively
## What changes were proposed in this pull request?
#15140 exposed ```JavaSparkContext.addFile(path: String, recursive: Boolean)``` to Python/R, then we can update SparkR ```spark.addFile``` to support adding directory recursively.

## How was this patch tested?
Added unit test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #15216 from yanboliang/spark-17577-2.
2016-09-26 16:47:57 -07:00
Andrew Mills 00be16df64 [Docs] Update spark-standalone.md to fix link
Corrected a link to the configuration.html page, it was pointing to a page that does not exist (configurations.html).

Documentation change, verified in preview.

Author: Andrew Mills <ammills01@users.noreply.github.com>

Closes #15244 from ammills01/master.
2016-09-26 16:41:14 -04:00
Sameer Agarwal 7c7586aef9 [SPARK-17652] Fix confusing exception message while reserving capacity
## What changes were proposed in this pull request?

This minor patch fixes a confusing exception message while reserving additional capacity in the vectorized parquet reader.

## How was this patch tested?

Exisiting Unit Tests

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #15225 from sameeragarwal/error-msg.
2016-09-26 13:21:08 -07:00
Liang-Chi Hsieh 8135e0e5eb [SPARK-17153][SQL] Should read partition data when reading new files in filestream without globbing
## What changes were proposed in this pull request?

When reading file stream with non-globbing path, the results return data with all `null`s for the
partitioned columns. E.g.,

    case class A(id: Int, value: Int)
    val data = spark.createDataset(Seq(
      A(1, 1),
      A(2, 2),
      A(2, 3))
    )
    val url = "/tmp/test"
    data.write.partitionBy("id").parquet(url)
    spark.read.parquet(url).show

    +-----+---+
    |value| id|
    +-----+---+
    |    2|  2|
    |    3|  2|
    |    1|  1|
    +-----+---+

    val s = spark.readStream.schema(spark.read.load(url).schema).parquet(url)
    s.writeStream.queryName("test").format("memory").start()

    sql("SELECT * FROM test").show

    +-----+----+
    |value|  id|
    +-----+----+
    |    2|null|
    |    3|null|
    |    1|null|
    +-----+----+

## How was this patch tested?

Jenkins tests.

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

Closes #14803 from viirya/filestreamsource-option.
2016-09-26 13:07:11 -07:00
Shixiong Zhu bde85f8b70 [SPARK-17649][CORE] Log how many Spark events got dropped in LiveListenerBus
## What changes were proposed in this pull request?

Log how many Spark events got dropped in LiveListenerBus so that the user can get insights on how to set a correct event queue size.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15220 from zsxwing/SPARK-17649.
2016-09-26 10:44:35 -07:00
hyukjinkwon f234b7cd79 [SPARK-16356][ML] Add testImplicits for ML unit tests and promote toDF()
## What changes were proposed in this pull request?

This was suggested in 101663f1ae (commitcomment-17114968).

This PR adds `testImplicits` to `MLlibTestSparkContext` so that some implicits such as `toDF()` can be sued across ml tests.

This PR also changes all the usages of `spark.createDataFrame( ... )` to `toDF()` where applicable in ml tests in Scala.

## How was this patch tested?

Existing tests should work.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14035 from HyukjinKwon/minor-ml-test.
2016-09-26 04:19:39 -07:00
Justin Pihony 50b89d05b7
[SPARK-14525][SQL] Make DataFrameWrite.save work for jdbc
## What changes were proposed in this pull request?

This change modifies the implementation of DataFrameWriter.save such that it works with jdbc, and the call to jdbc merely delegates to save.

## How was this patch tested?

This was tested via unit tests in the JDBCWriteSuite, of which I added one new test to cover this scenario.

## Additional details

rxin This seems to have been most recently touched by you and was also commented on in the JIRA.

This contribution is my original work and I license the work to the project under the project's open source license.

Author: Justin Pihony <justin.pihony@gmail.com>
Author: Justin Pihony <justin.pihony@typesafe.com>

Closes #12601 from JustinPihony/jdbc_reconciliation.
2016-09-26 09:54:22 +01:00
Yanbo Liang ac65139be9
[SPARK-17017][FOLLOW-UP][ML] Refactor of ChiSqSelector and add ML Python API.
## What changes were proposed in this pull request?
#14597 modified ```ChiSqSelector``` to support ```fpr``` type selector, however, it left some issue need to be addressed:
* We should allow users to set selector type explicitly rather than switching them by using different setting function, since the setting order will involves some unexpected issue. For example, if users both set ```numTopFeatures``` and ```percentile```, it will train ```kbest``` or ```percentile``` model based on the order of setting (the latter setting one will be trained). This make users confused, and we should allow users to set selector type explicitly. We handle similar issues at other place of ML code base such as ```GeneralizedLinearRegression``` and ```LogisticRegression```.
* Meanwhile, if there are more than one parameter except ```alpha``` can be set for ```fpr``` model, we can not handle it elegantly in the existing framework. And similar issues for ```kbest``` and ```percentile``` model. Setting selector type explicitly can solve this issue also.
* If setting selector type explicitly by users is allowed, we should handle param interaction such as if users set ```selectorType = percentile``` and ```alpha = 0.1```, we should notify users the parameter ```alpha``` will take no effect. We should handle complex parameter interaction checks at ```transformSchema```. (FYI #11620)
* We should use lower case of the selector type names to follow MLlib convention.
* Add ML Python API.

## How was this patch tested?
Unit test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #15214 from yanboliang/spark-17017.
2016-09-26 09:45:33 +01:00
Burak Yavuz 59d87d2407 [SPARK-17650] malformed url's throw exceptions before bricking Executors
## What changes were proposed in this pull request?

When a malformed URL was sent to Executors through `sc.addJar` and `sc.addFile`, the executors become unusable, because they constantly throw `MalformedURLException`s and can never acknowledge that the file or jar is just bad input.

This PR tries to fix that problem by making sure MalformedURLs can never be submitted through `sc.addJar` and `sc.addFile`. Another solution would be to blacklist bad files and jars on Executors. Maybe fail the first time, and then ignore the second time (but print a warning message).

## How was this patch tested?

Unit tests in SparkContextSuite

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15224 from brkyvz/SPARK-17650.
2016-09-25 22:57:31 -07:00
xin wu de333d121d [SPARK-17551][SQL] Add DataFrame API for null ordering
## What changes were proposed in this pull request?
This pull request adds Scala/Java DataFrame API for null ordering (NULLS FIRST | LAST).

Also did some minor clean up for related code (e.g. incorrect indentation), and renamed "orderby-nulls-ordering.sql" to be consistent with existing test files.

## How was this patch tested?
Added a new test case in DataFrameSuite.

Author: petermaxlee <petermaxlee@gmail.com>
Author: Xin Wu <xinwu@us.ibm.com>

Closes #15123 from petermaxlee/SPARK-17551.
2016-09-25 16:46:12 -07:00
Yanbo Liang 7945daed12 [MINOR][SPARKR] Add sparkr-vignettes.html to gitignore.
## What changes were proposed in this pull request?
Add ```sparkr-vignettes.html``` to ```.gitignore```.

## How was this patch tested?
No need test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #15215 from yanboliang/ignore.
2016-09-24 01:03:11 -07:00
Sean Owen 248916f558
[SPARK-17057][ML] ProbabilisticClassifierModels' thresholds should have at most one 0
## What changes were proposed in this pull request?

Match ProbabilisticClassifer.thresholds requirements to R randomForest cutoff, requiring all > 0

## How was this patch tested?

Jenkins tests plus new test cases

Author: Sean Owen <sowen@cloudera.com>

Closes #15149 from srowen/SPARK-17057.
2016-09-24 08:15:55 +01:00
Sean Owen f3fe55439e
[SPARK-10835][ML] Word2Vec should accept non-null string array, in addition to existing null string array
## What changes were proposed in this pull request?

To match Tokenizer and for compatibility with Word2Vec, output a nullable string array type in NGram

## How was this patch tested?

Jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15179 from srowen/SPARK-10835.
2016-09-24 08:06:41 +01:00
Shivaram Venkataraman 7c382524a9 [SPARK-17651][SPARKR] Set R package version number along with mvn
## What changes were proposed in this pull request?

This PR sets the R package version while tagging releases. Note that since R doesn't accept `-SNAPSHOT` in version number field, we remove that while setting the next version

## How was this patch tested?

Tested manually by running locally

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #15223 from shivaram/sparkr-version-change.
2016-09-23 14:35:18 -07:00
jisookim 90a30f4634 [SPARK-12221] add cpu time to metrics
Currently task metrics don't support executor CPU time, so there's no way to calculate how much CPU time a stage/task took from History Server metrics. This PR enables reporting CPU time.

Author: jisookim <jisookim0513@gmail.com>

Closes #10212 from jisookim0513/add-cpu-time-metric.
2016-09-23 13:43:47 -07:00
Michael Armbrust 988c714573 [SPARK-17643] Remove comparable requirement from Offset
For some sources, it is difficult to provide a global ordering based only on the data in the offset.  Since we don't use comparison for correctness, lets remove it.

Author: Michael Armbrust <michael@databricks.com>

Closes #15207 from marmbrus/removeComparable.
2016-09-23 12:17:59 -07:00
Jeff Zhang f62ddc5983 [SPARK-17210][SPARKR] sparkr.zip is not distributed to executors when running sparkr in RStudio
## What changes were proposed in this pull request?

Spark will add sparkr.zip to archive only when it is yarn mode (SparkSubmit.scala).
```
    if (args.isR && clusterManager == YARN) {
      val sparkRPackagePath = RUtils.localSparkRPackagePath
      if (sparkRPackagePath.isEmpty) {
        printErrorAndExit("SPARK_HOME does not exist for R application in YARN mode.")
      }
      val sparkRPackageFile = new File(sparkRPackagePath.get, SPARKR_PACKAGE_ARCHIVE)
      if (!sparkRPackageFile.exists()) {
        printErrorAndExit(s"$SPARKR_PACKAGE_ARCHIVE does not exist for R application in YARN mode.")
      }
      val sparkRPackageURI = Utils.resolveURI(sparkRPackageFile.getAbsolutePath).toString

      // Distribute the SparkR package.
      // Assigns a symbol link name "sparkr" to the shipped package.
      args.archives = mergeFileLists(args.archives, sparkRPackageURI + "#sparkr")

      // Distribute the R package archive containing all the built R packages.
      if (!RUtils.rPackages.isEmpty) {
        val rPackageFile =
          RPackageUtils.zipRLibraries(new File(RUtils.rPackages.get), R_PACKAGE_ARCHIVE)
        if (!rPackageFile.exists()) {
          printErrorAndExit("Failed to zip all the built R packages.")
        }

        val rPackageURI = Utils.resolveURI(rPackageFile.getAbsolutePath).toString
        // Assigns a symbol link name "rpkg" to the shipped package.
        args.archives = mergeFileLists(args.archives, rPackageURI + "#rpkg")
      }
    }
```
So it is necessary to pass spark.master from R process to JVM. Otherwise sparkr.zip won't be distributed to executor.  Besides that I also pass spark.yarn.keytab/spark.yarn.principal to spark side, because JVM process need them to access secured cluster.

## How was this patch tested?

Verify it manually in R Studio using the following code.
```
Sys.setenv(SPARK_HOME="/Users/jzhang/github/spark")
.libPaths(c(file.path(Sys.getenv(), "R", "lib"), .libPaths()))
library(SparkR)
sparkR.session(master="yarn-client", sparkConfig = list(spark.executor.instances="1"))
df <- as.DataFrame(mtcars)
head(df)

```

…

Author: Jeff Zhang <zjffdu@apache.org>

Closes #14784 from zjffdu/SPARK-17210.
2016-09-23 11:37:43 -07:00
WeichenXu f89808b0fd [SPARK-17499][SPARKR][ML][MLLIB] make the default params in sparkR spark.mlp consistent with MultilayerPerceptronClassifier
## What changes were proposed in this pull request?

update `MultilayerPerceptronClassifierWrapper.fit` paramter type:
`layers: Array[Int]`
`seed: String`

update several default params in sparkR `spark.mlp`:
`tol` --> 1e-6
`stepSize` --> 0.03
`seed` --> NULL ( when seed == NULL, the scala-side wrapper regard it as a `null` value and the seed will use the default one )
r-side `seed` only support 32bit integer.

remove `layers` default value, and move it in front of those parameters with default value.
add `layers` parameter validation check.

## How was this patch tested?

tests added.

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #15051 from WeichenXu123/update_py_mlp_default.
2016-09-23 11:14:22 -07:00
Holden Karau 90d5754212
[SPARK-16861][PYSPARK][CORE] Refactor PySpark accumulator API on top of Accumulator V2
## What changes were proposed in this pull request?

Move the internals of the PySpark accumulator API from the old deprecated API on top of the new accumulator API.

## How was this patch tested?

The existing PySpark accumulator tests (both unit tests and doc tests at the start of accumulator.py).

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

Closes #14467 from holdenk/SPARK-16861-refactor-pyspark-accumulator-api.
2016-09-23 09:44:30 +01:00
hyukjinkwon 5c5396cb47
[BUILD] Closes some stale PRs
## What changes were proposed in this pull request?

This PR proposes to close some stale PRs and ones suggested to be closed by committer(s)

Closes #12415
Closes #14765
Closes #15118
Closes #15184
Closes #15183
Closes #9440
Closes #15023
Closes #14643
Closes #14827

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15198 from HyukjinKwon/stale-prs.
2016-09-23 09:41:50 +01:00
Shixiong Zhu 62ccf27ab4 [SPARK-17640][SQL] Avoid using -1 as the default batchId for FileStreamSource.FileEntry
## What changes were proposed in this pull request?

Avoid using -1 as the default batchId for FileStreamSource.FileEntry so that we can make sure not writing any FileEntry(..., batchId = -1) into the log. This also avoids people misusing it in future (#15203 is an example).

## How was this patch tested?

Jenkins.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15206 from zsxwing/cleanup.
2016-09-22 23:35:08 -07:00
Joseph K. Bradley 947b8c6e3a [SPARK-16719][ML] Random Forests should communicate fewer trees on each iteration
## What changes were proposed in this pull request?

RandomForest currently sends the entire forest to each worker on each iteration. This is because (a) the node queue is FIFO and (b) the closure references the entire array of trees (topNodes). (a) causes RFs to handle splits in many trees, especially early on in learning. (b) sends all trees explicitly.

This PR:
(a) Change the RF node queue to be FILO (a stack), so that RFs tend to focus on 1 or a few trees before focusing on others.
(b) Change topNodes to pass only the trees required on that iteration.

## How was this patch tested?

Unit tests:
* Existing tests for correctness of tree learning
* Manually modifying code and running tests to verify that a small number of trees are communicated on each iteration
  * This last item is hard to test via unit tests given the current APIs.

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #14359 from jkbradley/rfs-fewer-trees.
2016-09-22 22:27:28 -07:00
Marcelo Vanzin a4aeb7677b [SPARK-17639][BUILD] Add jce.jar to buildclasspath when building.
This was missing, preventing code that uses javax.crypto to properly
compile in Spark.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15204 from vanzin/SPARK-17639.
2016-09-22 21:35:25 -07:00
Yucai Yu 79159a1e87 [SPARK-17635][SQL] Remove hardcode "agg_plan" in HashAggregateExec
## What changes were proposed in this pull request?

"agg_plan" are hardcoded in HashAggregateExec, which have potential issue, so removing them.

## How was this patch tested?

existing tests.

Author: Yucai Yu <yucai.yu@intel.com>

Closes #15199 from yucai/agg_plan.
2016-09-22 17:22:56 -07:00
Burak Yavuz a166196831 [SPARK-17569][SPARK-17569][TEST] Make the unit test added for work again
## What changes were proposed in this pull request?

A [PR](a6aade0042) was merged concurrently that made the unit test for PR #15122 not test anything anymore. This PR fixes the test.

## How was this patch tested?

Changed line 0d63487502/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala (L137)
from `false` to `true` and made sure the unit test failed.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15203 from brkyvz/fix-test.
2016-09-22 16:50:22 -07:00
Gayathri Murali f4f6bd8c98 [SPARK-16240][ML] ML persistence backward compatibility for LDA
## What changes were proposed in this pull request?

Allow Spark 2.x to load instances of LDA, LocalLDAModel, and DistributedLDAModel saved from Spark 1.6.

## How was this patch tested?

I tested this manually, saving the 3 types from 1.6 and loading them into master (2.x).  In the future, we can add generic tests for testing backwards compatibility across all ML models in SPARK-15573.

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #15034 from jkbradley/lda-backwards.
2016-09-22 16:34:42 -07:00
Herman van Hovell 0d63487502 [SPARK-17616][SQL] Support a single distinct aggregate combined with a non-partial aggregate
## What changes were proposed in this pull request?
We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example:
```sql
select   grp,
         collect_list(col1),
         count(distinct col2)
from     tbl_a
group by 1
```
This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6).

## How was this patch tested?
Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`.

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

Closes #15187 from hvanhovell/SPARK-17616.
2016-09-22 14:29:27 -07:00
Shixiong Zhu 3cdae0ff2f [SPARK-17638][STREAMING] Stop JVM StreamingContext when the Python process is dead
## What changes were proposed in this pull request?

When the Python process is dead, the JVM StreamingContext is still running. Hence we will see a lot of Py4jException before the JVM process exits. It's better to stop the JVM StreamingContext to avoid those annoying logs.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15201 from zsxwing/stop-jvm-ssc.
2016-09-22 14:26:45 -07:00
Burak Yavuz 85d609cf25 [SPARK-17613] S3A base paths with no '/' at the end return empty DataFrames
## What changes were proposed in this pull request?

Consider you have a bucket as `s3a://some-bucket`
and under it you have files:
```
s3a://some-bucket/file1.parquet
s3a://some-bucket/file2.parquet
```
Getting the parent path of `s3a://some-bucket/file1.parquet` yields
`s3a://some-bucket/` and the ListingFileCatalog uses this as the key in the hash map.

When catalog.allFiles is called, we use `s3a://some-bucket` (no slash at the end) to get the list of files, and we're left with an empty list!

This PR fixes this by adding a `/` at the end of the `URI` iff the given `Path` doesn't have a parent, i.e. is the root. This is a no-op if the path already had a `/` at the end, and is handled through the Hadoop Path, path merging semantics.

## How was this patch tested?

Unit test in `FileCatalogSuite`.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15169 from brkyvz/SPARK-17613.
2016-09-22 13:05:41 -07:00
Shivaram Venkataraman 9f24a17c59 Skip building R vignettes if Spark is not built
## What changes were proposed in this pull request?

When we build the docs separately we don't have the JAR files from the Spark build in
the same tree. As the SparkR vignettes need to launch a SparkContext to be built, we skip building them if JAR files don't exist

## How was this patch tested?

To test this we can run the following:
```
build/mvn -DskipTests -Psparkr clean
./R/create-docs.sh
```
You should see a line `Skipping R vignettes as Spark JARs not found` at the end

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #15200 from shivaram/sparkr-vignette-skip.
2016-09-22 11:52:42 -07:00
Dhruve Ashar 17b72d31e0 [SPARK-17365][CORE] Remove/Kill multiple executors together to reduce RPC call time.
## What changes were proposed in this pull request?
We are killing multiple executors together instead of iterating over expensive RPC calls to kill single executor.

## How was this patch tested?
Executed sample spark job to observe executors being killed/removed with dynamic allocation enabled.

Author: Dhruve Ashar <dashar@yahoo-inc.com>
Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #15152 from dhruve/impr/SPARK-17365.
2016-09-22 10:10:37 -07:00
Wenchen Fan 8a02410a92 [SQL][MINOR] correct the comment of SortBasedAggregationIterator.safeProj
## What changes were proposed in this pull request?

This comment went stale long time ago, this PR fixes it according to my understanding.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15095 from cloud-fan/update-comment.
2016-09-22 23:25:32 +08:00
WeichenXu 72d9fba26c [SPARK-17281][ML][MLLIB] Add treeAggregateDepth parameter for AFTSurvivalRegression
## What changes were proposed in this pull request?

Add treeAggregateDepth parameter for AFTSurvivalRegression to keep consistent with LiR/LoR.

## How was this patch tested?

Existing tests.

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #14851 from WeichenXu123/add_treeAggregate_param_for_survival_regression.
2016-09-22 04:35:54 -07:00
frreiss 646f383465
[SPARK-17421][DOCS] Documenting the current treatment of MAVEN_OPTS.
## What changes were proposed in this pull request?

Modified the documentation to clarify that `build/mvn` and `pom.xml` always add Java 7-specific parameters to `MAVEN_OPTS`, and that developers can safely ignore warnings about `-XX:MaxPermSize` that may result from compiling or running tests with Java 8.

## How was this patch tested?

Rebuilt HTML documentation, made sure that building-spark.html displays correctly in a browser.

Author: frreiss <frreiss@us.ibm.com>

Closes #15005 from frreiss/fred-17421a.
2016-09-22 10:31:15 +01:00
Zhenhua Wang de7df7defc [SPARK-17625][SQL] set expectedOutputAttributes when converting SimpleCatalogRelation to LogicalRelation
## What changes were proposed in this pull request?

We should set expectedOutputAttributes when converting SimpleCatalogRelation to LogicalRelation, otherwise the outputs of LogicalRelation are different from outputs of SimpleCatalogRelation - they have different exprId's.

## How was this patch tested?

add a test case

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #15182 from wzhfy/expectedAttributes.
2016-09-22 14:48:49 +08:00
gatorsmile 3a80f92f8f [SPARK-17492][SQL] Fix Reading Cataloged Data Sources without Extending SchemaRelationProvider
### What changes were proposed in this pull request?
For data sources without extending `SchemaRelationProvider`, we expect users to not specify schemas when they creating tables. If the schema is input from users, an exception is issued.

Since Spark 2.1, for any data source, to avoid infer the schema every time, we store the schema in the metastore catalog. Thus, when reading a cataloged data source table, the schema could be read from metastore catalog. In this case, we also got an exception. For example,

```Scala
sql(
  s"""
     |CREATE TABLE relationProvierWithSchema
     |USING org.apache.spark.sql.sources.SimpleScanSource
     |OPTIONS (
     |  From '1',
     |  To '10'
     |)
   """.stripMargin)
spark.table(tableName).show()
```
```
org.apache.spark.sql.sources.SimpleScanSource does not allow user-specified schemas.;
```

This PR is to fix the above issue. When building a data source, we introduce a flag `isSchemaFromUsers` to indicate whether the schema is really input from users. If true, we issue an exception. Otherwise, we will call the `createRelation` of `RelationProvider` to generate the `BaseRelation`, in which it contains the actual schema.

### How was this patch tested?
Added a few cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15046 from gatorsmile/tempViewCases.
2016-09-22 13:19:06 +08:00
Yadong Qi cb324f6115 [SPARK-17425][SQL] Override sameResult in HiveTableScanExec to make ReuseExchange work in text format table
## What changes were proposed in this pull request?
The PR will override the `sameResult` in `HiveTableScanExec` to make `ReuseExchange` work in text format table.

## How was this patch tested?
# SQL
```sql
SELECT * FROM src t1
JOIN src t2 ON t1.key = t2.key
JOIN src t3 ON t1.key = t3.key;
```

# Before
```
== Physical Plan ==
*BroadcastHashJoin [key#30], [key#34], Inner, BuildRight
:- *BroadcastHashJoin [key#30], [key#32], Inner, BuildRight
:  :- *Filter isnotnull(key#30)
:  :  +- HiveTableScan [key#30, value#31], MetastoreRelation default, src
:  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
:     +- *Filter isnotnull(key#32)
:        +- HiveTableScan [key#32, value#33], MetastoreRelation default, src
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
   +- *Filter isnotnull(key#34)
      +- HiveTableScan [key#34, value#35], MetastoreRelation default, src
```

# After
```
== Physical Plan ==
*BroadcastHashJoin [key#2], [key#6], Inner, BuildRight
:- *BroadcastHashJoin [key#2], [key#4], Inner, BuildRight
:  :- *Filter isnotnull(key#2)
:  :  +- HiveTableScan [key#2, value#3], MetastoreRelation default, src
:  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
:     +- *Filter isnotnull(key#4)
:        +- HiveTableScan [key#4, value#5], MetastoreRelation default, src
+- ReusedExchange [key#6, value#7], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
```

cc: davies cloud-fan

Author: Yadong Qi <qiyadong2010@gmail.com>

Closes #14988 from watermen/SPARK-17425.
2016-09-22 13:04:42 +08:00
Wenchen Fan b50b34f561 [SPARK-17609][SQL] SessionCatalog.tableExists should not check temp view
## What changes were proposed in this pull request?

After #15054 , there is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks.

This PR also improves the `getTempViewOrPermanentTableMetadata` that is introduced in  #15054 , to make the code simpler.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15160 from cloud-fan/exists.
2016-09-22 12:52:09 +08:00
Davies Liu 8bde03bf9a [SPARK-17494][SQL] changePrecision() on compact decimal should respect rounding mode
## What changes were proposed in this pull request?

Floor()/Ceil() of decimal is implemented using changePrecision() by passing a rounding mode, but the rounding mode is not respected when the decimal is in compact mode (could fit within a Long).

This Update the changePrecision() to respect rounding mode, which could be ROUND_FLOOR, ROUND_CEIL, ROUND_HALF_UP, ROUND_HALF_EVEN.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #15154 from davies/decimal_round.
2016-09-21 21:02:30 -07:00
Michael Armbrust 3497ebe511 [SPARK-17627] Mark Streaming Providers Experimental
All of structured streaming is experimental in its first release.  We missed the annotation on two of the APIs.

Author: Michael Armbrust <michael@databricks.com>

Closes #15188 from marmbrus/experimentalApi.
2016-09-21 20:59:46 -07:00
Yanbo Liang 6902edab7e [SPARK-17315][FOLLOW-UP][SPARKR][ML] Fix print of Kolmogorov-Smirnov test summary
## What changes were proposed in this pull request?
#14881 added Kolmogorov-Smirnov Test wrapper to SparkR. I found that ```print.summary.KSTest``` was implemented inappropriately and result in no effect.
Running the following code for KSTest:
```Scala
data <- data.frame(test = c(0.1, 0.15, 0.2, 0.3, 0.25, -1, -0.5))
df <- createDataFrame(data)
testResult <- spark.kstest(df, "test", "norm")
summary(testResult)
```
Before this PR:
![image](https://cloud.githubusercontent.com/assets/1962026/18615016/b9a2823a-7d4f-11e6-934b-128beade355e.png)
After this PR:
![image](https://cloud.githubusercontent.com/assets/1962026/18615014/aafe2798-7d4f-11e6-8b99-c705bb9fe8f2.png)
The new implementation is similar with [```print.summary.GeneralizedLinearRegressionModel```](https://github.com/apache/spark/blob/master/R/pkg/R/mllib.R#L284) of SparkR and [```print.summary.glm```](https://svn.r-project.org/R/trunk/src/library/stats/R/glm.R) of native R.

BTW, I removed the comparison of ```print.summary.KSTest``` in unit test, since it's only wrappers of the summary output which has been checked. Another reason is that these comparison will output summary information to the test console, it will make the test output in a mess.

## How was this patch tested?
Existing test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #15139 from yanboliang/spark-17315.
2016-09-21 20:14:18 -07:00
Yanbo Liang c133907c5d [SPARK-17577][SPARKR][CORE] SparkR support add files to Spark job and get by executors
## What changes were proposed in this pull request?
Scala/Python users can add files to Spark job by submit options ```--files``` or ```SparkContext.addFile()```. Meanwhile, users can get the added file by ```SparkFiles.get(filename)```.
We should also support this function for SparkR users, since they also have the requirements for some shared dependency files. For example, SparkR users can download third party R packages to driver firstly, add these files to the Spark job as dependency by this API and then each executor can install these packages by ```install.packages```.

## How was this patch tested?
Add unit test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #15131 from yanboliang/spark-17577.
2016-09-21 20:08:28 -07:00
Burak Yavuz 7cbe216449 [SPARK-17569] Make StructuredStreaming FileStreamSource batch generation faster
## What changes were proposed in this pull request?

While getting the batch for a `FileStreamSource` in StructuredStreaming, we know which files we must take specifically. We already have verified that they exist, and have committed them to a metadata log. When creating the FileSourceRelation however for an incremental execution, the code checks the existence of every single file once again!

When you have 100,000s of files in a folder, creating the first batch takes 2 hours+ when working with S3! This PR disables that check

## How was this patch tested?

Added a unit test to `FileStreamSource`.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15122 from brkyvz/SPARK-17569.
2016-09-21 17:12:52 -07:00
jerryshao 8c3ee2bc42 [SPARK-17512][CORE] Avoid formatting to python path for yarn and mesos cluster mode
## What changes were proposed in this pull request?

Yarn and mesos cluster mode support remote python path (HDFS/S3 scheme) by their own mechanism, it is not necessary to check and format the python when running on these modes. This is a potential regression compared to 1.6, so here propose to fix it.

## How was this patch tested?

Unit test to verify SparkSubmit arguments, also with local cluster verification. Because of lack of `MiniDFSCluster` support in Spark unit test, there's no integration test added.

Author: jerryshao <sshao@hortonworks.com>

Closes #15137 from jerryshao/SPARK-17512.
2016-09-21 17:57:21 -04:00
Imran Rashid 9fcf1c51d5 [SPARK-17623][CORE] Clarify type of TaskEndReason with a failed task.
## What changes were proposed in this pull request?

In TaskResultGetter, enqueueFailedTask currently deserializes the result
as a TaskEndReason. But the type is actually more specific, its a
TaskFailedReason. This just leads to more blind casting later on – it
would be more clear if the msg was cast to the right type immediately,
so method parameter types could be tightened.

## How was this patch tested?

Existing unit tests via jenkins.  Note that the code was already performing a blind-cast to a TaskFailedReason before in any case, just in a different spot, so there shouldn't be any behavior change.

Author: Imran Rashid <irashid@cloudera.com>

Closes #15181 from squito/SPARK-17623.
2016-09-21 17:49:36 -04:00
Marcelo Vanzin 2cd1bfa4f0 [SPARK-4563][CORE] Allow driver to advertise a different network address.
The goal of this feature is to allow the Spark driver to run in an
isolated environment, such as a docker container, and be able to use
the host's port forwarding mechanism to be able to accept connections
from the outside world.

The change is restricted to the driver: there is no support for achieving
the same thing on executors (or the YARN AM for that matter). Those still
need full access to the outside world so that, for example, connections
can be made to an executor's block manager.

The core of the change is simple: add a new configuration that tells what's
the address the driver should bind to, which can be different than the address
it advertises to executors (spark.driver.host). Everything else is plumbing
the new configuration where it's needed.

To use the feature, the host starting the container needs to set up the
driver's port range to fall into a range that is being forwarded; this
required the block manager port to need a special configuration just for
the driver, which falls back to the existing spark.blockManager.port when
not set. This way, users can modify the driver settings without affecting
the executors; it would theoretically be nice to also have different
retry counts for driver and executors, but given that docker (at least)
allows forwarding port ranges, we can probably live without that for now.

Because of the nature of the feature it's kinda hard to add unit tests;
I just added a simple one to make sure the configuration works.

This was tested with a docker image running spark-shell with the following
command:

 docker blah blah blah \
   -p 38000-38100:38000-38100 \
   [image] \
   spark-shell \
     --num-executors 3 \
     --conf spark.shuffle.service.enabled=false \
     --conf spark.dynamicAllocation.enabled=false \
     --conf spark.driver.host=[host's address] \
     --conf spark.driver.port=38000 \
     --conf spark.driver.blockManager.port=38020 \
     --conf spark.ui.port=38040

Running on YARN; verified the driver works, executors start up and listen
on ephemeral ports (instead of using the driver's config), and that caching
and shuffling (without the shuffle service) works. Clicked through the UI
to make sure all pages (including executor thread dumps) worked. Also tested
apps without docker, and ran unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15120 from vanzin/SPARK-4563.
2016-09-21 14:42:41 -07:00
Sean Owen b4a4421b61 [SPARK-11918][ML] Better error from WLS for cases like singular input
## What changes were proposed in this pull request?

Update error handling for Cholesky decomposition to provide a little more info when input is singular.

## How was this patch tested?

New test case; jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15177 from srowen/SPARK-11918.
2016-09-21 18:56:16 +00:00
Josh Rosen d7ee12211a [SPARK-17418] Prevent kinesis-asl-assembly artifacts from being published
This patch updates the `kinesis-asl-assembly` build to prevent that module from being published as part of Maven releases and snapshot builds.

The `kinesis-asl-assembly` includes classes from the Kinesis Client Library (KCL) and Kinesis Producer Library (KPL), both of which are licensed under the Amazon Software License and are therefore prohibited from being distributed in Apache releases.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15167 from JoshRosen/stop-publishing-kinesis-assembly.
2016-09-21 11:38:10 -07:00
Liang-Chi Hsieh 248922fd4f [SPARK-17590][SQL] Analyze CTE definitions at once and allow CTE subquery to define CTE
## What changes were proposed in this pull request?

We substitute logical plan with CTE definitions in the analyzer rule CTESubstitution. A CTE definition can be used in the logical plan for multiple times, and its analyzed logical plan should be the same. We should not analyze CTE definitions multiple times when they are reused in the query.

By analyzing CTE definitions before substitution, we can support defining CTE in subquery.

## How was this patch tested?

Jenkins tests.

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

Closes #15146 from viirya/cte-analysis-once.
2016-09-21 06:53:42 -07:00