aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-19155][ML] Make family case insensitive in GLMactuaryzhang2017-01-232-4/+6
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is a supplement to PR #16516 which did not make the value from `getFamily` case insensitive. Current tests of poisson/binomial glm with weight fail when specifying 'Poisson' or 'Binomial', because the calculation of `dispersion` and `pValue` checks the value of family retrieved from `getFamily` ``` model.getFamily == Binomial.name || model.getFamily == Poisson.name ``` ## How was this patch tested? Update existing tests for 'Poisson' and 'Binomial'. yanboliang felixcheung imatiach-msft Author: actuaryzhang <actuaryzhang10@gmail.com> Closes #16675 from actuaryzhang/family.
* [SPARK-19309][SQL] disable common subexpression elimination for conditional ↵Wenchen Fan2017-01-237-171/+84
| | | | | | | | | | | | | | | | | | | | expressions ## What changes were proposed in this pull request? As I pointed out in https://github.com/apache/spark/pull/15807#issuecomment-259143655 , the current subexpression elimination framework has a problem, it always evaluates all common subexpressions at the beginning, even they are inside conditional expressions and may not be accessed. Ideally we should implement it like scala lazy val, so we only evaluate it when it gets accessed at lease once. https://github.com/apache/spark/issues/15837 tries this approach, but it seems too complicated and may introduce performance regression. This PR simply stops common subexpression elimination for conditional expressions, with some cleanup. ## How was this patch tested? regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16659 from cloud-fan/codegen.
* [SPARK-19229][SQL] Disallow Creating Hive Source Tables when Hive Support is ↵gatorsmile2017-01-2218-93/+72
| | | | | | | | | | | | | | Not Enabled ### What changes were proposed in this pull request? It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables. ### How was this patch tested? Fixed the test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #16587 from gatorsmile/blockHiveTable.
* [SPARK-16101][SQL] Refactoring CSV read path to be consistent with JSON data ↵hyukjinkwon2017-01-237-316/+293
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | source ## What changes were proposed in this pull request? This PR refactors CSV read path to be consistent with JSON data source. It makes the methods in classes have consistent arguments with JSON ones. `UnivocityParser` and `JacksonParser` ``` scala private[csv] class UnivocityParser( schema: StructType, requiredSchema: StructType, options: CSVOptions) extends Logging { ... def parse(input: String): Seq[InternalRow] = { ... ``` ``` scala class JacksonParser( schema: StructType, columnNameOfCorruptRecord: String, options: JSONOptions) extends Logging { ... def parse(input: String): Option[InternalRow] = { ... ``` These allow parsing an iterator (`String` to `InternalRow`) as below for both JSON and CSV: ```scala iter.flatMap(parser.parse) ``` ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16669 from HyukjinKwon/SPARK-16101-read.
* [SPARK-19291][SPARKR][ML] spark.gaussianMixture supports output log-likelihood.Yanbo Liang2017-01-213-5/+19
| | | | | | | | | | | | ## What changes were proposed in this pull request? ```spark.gaussianMixture``` supports output total log-likelihood for the model like R ```mvnormalmixEM```. ## How was this patch tested? R unit test. Author: Yanbo Liang <ybliang8@gmail.com> Closes #16646 from yanboliang/spark-19291.
* [SPARK-19155][ML] MLlib GeneralizedLinearRegression family and link should ↵Yanbo Liang2017-01-212-6/+6
| | | | | | | | | | | | | | case insensitive ## What changes were proposed in this pull request? MLlib ```GeneralizedLinearRegression``` ```family``` and ```link``` should be case insensitive. This is consistent with some other MLlib params such as [```featureSubsetStrategy```](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala#L415). ## How was this patch tested? Update corresponding tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #16516 from yanboliang/spark-19133.
* [SPARK-19153][SQL] DataFrameWriter.saveAsTable work with create partitioned ↵windpiger2017-01-223-19/+26
| | | | | | | | | | | | | | | | | table ## What changes were proposed in this pull request? After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19153), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support. this PR provide DataFrameWriter.saveAsTable work with hive format to create partitioned table. ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16593 from windpiger/saveAsTableWithPartitionedTable.
* [SPARK-19117][SPARK-18922][TESTS] Fix the rest of flaky, newly introduced ↵hyukjinkwon2017-01-215-9/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | and missed test failures on Windows ## What changes were proposed in this pull request? **Failed tests** ``` org.apache.spark.sql.hive.execution.HiveQuerySuite: - transform with SerDe3 *** FAILED *** - transform with SerDe4 *** FAILED *** ``` ``` org.apache.spark.sql.hive.execution.HiveDDLSuite: - create hive serde table with new syntax *** FAILED *** - add/drop partition with location - managed table *** FAILED *** ``` ``` org.apache.spark.sql.hive.ParquetMetastoreSuite: - Explicitly added partitions should be readable after load *** FAILED *** - Non-partitioned table readable after load *** FAILED *** ``` **Aborted tests** ``` Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.HiveSerDeSuite *** ABORTED *** (157 milliseconds) org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilessales.txt; ``` **Flaky tests(failed 9ish out of 10)** ``` org.apache.spark.scheduler.SparkListenerSuite: - local metrics *** FAILED *** ``` ## How was this patch tested? Manually tested via AppVeyor. **Failed tests** ``` org.apache.spark.sql.hive.execution.HiveQuerySuite: - transform with SerDe3 !!! CANCELED !!! (0 milliseconds) - transform with SerDe4 !!! CANCELED !!! (0 milliseconds) ``` ``` org.apache.spark.sql.hive.execution.HiveDDLSuite: - create hive serde table with new syntax (1 second, 672 milliseconds) - add/drop partition with location - managed table (2 seconds, 391 milliseconds) ``` ``` org.apache.spark.sql.hive.ParquetMetastoreSuite: - Explicitly added partitions should be readable after load (609 milliseconds) - Non-partitioned table readable after load (344 milliseconds) ``` **Aborted tests** ``` spark.sql.hive.execution.HiveSerDeSuite: - Read with RegexSerDe (2 seconds, 142 milliseconds) - Read and write with LazySimpleSerDe (tab separated) (2 seconds) - Read with AvroSerDe (1 second, 47 milliseconds) - Read Partitioned with AvroSerDe (1 second, 422 milliseconds) ``` **Flaky tests (failed 9ish out of 10)** ``` org.apache.spark.scheduler.SparkListenerSuite: - local metrics (4 seconds, 562 milliseconds) ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16586 from HyukjinKwon/set-path-appveyor.
* [SPARK-17724][STREAMING][WEBUI] Unevaluated new lines in tooltip in DAG ↵Xin Ren2017-01-211-2/+32
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Visualization of a job https://issues.apache.org/jira/browse/SPARK-17724 ## What changes were proposed in this pull request? For unevaluated `\n`, evaluate it and enable line break, for Streaming WebUI `stages` page and `job` page. (I didn't change Scala source file, since Jetty server has to somehow indicate line break and js to code display it.) (This PR is a continue from previous PR https://github.com/apache/spark/pull/15353 for the same issue, sorry being so long time) Two changes: 1. RDD Node tooltipText is actually showing the `<circle>` `title` property, so I set extra attribute in `spark-dag-viz.js`: `.attr("data-html", "true")` `<circle x="-5" y="-5" r="5" data-toggle="tooltip" data-placement="bottom" title="" data-original-title="ParallelCollectionRDD [9]\nmakeRDD at QueueStream.scala:49"></circle>` 2. Static `<tspan>` text of each stage, split by `/n`, and append an extra `<tspan>` element to its parentNode `<text><tspan xml:space="preserve" dy="1em" x="1">reduceByKey</tspan><tspan xml:space="preserve" dy="1em" x="1">reduceByKey/n 23:34:49</tspan></text> ` ## UI changes Screenshot **before fix**, `\n` is not evaluated in both circle tooltipText and static text: ![screen shot 2017-01-19 at 12 21 54 am](https://cloud.githubusercontent.com/assets/3925641/22098829/53c7f49c-dddd-11e6-9daa-b3ddb6044114.png) Screenshot **after fix**: ![screen shot 2017-01-19 at 12 20 30 am](https://cloud.githubusercontent.com/assets/3925641/22098806/294910d4-dddd-11e6-9948-d942e09f545e.png) ## How was this patch tested? Tested locally. For Streaming WebUI `stages` page and `job` page, on multiple browsers: - Chrome - Firefox - Safari Author: Xin Ren <renxin.ubc@gmail.com> Closes #16643 from keypointt/SPARK-17724-2nd.
* [SPARK-19305][SQL] partitioned table should always put partition columns at ↵Wenchen Fan2017-01-212-18/+72
| | | | | | | | | | | | | | | | | | | | the end of table schema ## What changes were proposed in this pull request? For data source tables, we will always reorder the specified table schema, or the query in CTAS, to put partition columns at the end. e.g. `CREATE TABLE t(a int, b int, c int, d int) USING parquet PARTITIONED BY (d, b)` will create a table with schema `<a, c, d, b>` Hive serde tables don't have this problem before, because its CREATE TABLE syntax specifies data schema and partition schema individually. However, after we unifed the CREATE TABLE syntax, Hive serde table also need to do the reorder. This PR puts the reorder logic in a analyzer rule, which works with both data source tables and Hive serde tables. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16655 from cloud-fan/schema.
* [SPARK-14536][SQL] fix to handle null value in array type column for postgres.sureshthalamati2017-01-202-5/+13
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? JDBC read is failing with NPE due to missing null value check for array data type if the source table has null values in the array type column. For null values Resultset.getArray() returns null. This PR adds null safe check to the Resultset.getArray() value before invoking method on the Array object. ## How was this patch tested? Updated the PostgresIntegration test suite to test null values. Ran docker integration tests on my laptop. Author: sureshthalamati <suresh.thalamati@gmail.com> Closes #15192 from sureshthalamati/jdbc_array_null_fix-SPARK-14536.
* [SPARK-16101][SQL] Refactoring CSV write path to be consistent with JSON ↵hyukjinkwon2017-01-215-115/+135
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | data source ## What changes were proposed in this pull request? This PR refactors CSV write path to be consistent with JSON data source. This PR makes the methods in classes have consistent arguments with JSON ones. - `UnivocityGenerator` and `JacksonGenerator` ``` scala private[csv] class UnivocityGenerator( schema: StructType, writer: Writer, options: CSVOptions = new CSVOptions(Map.empty[String, String])) { ... def write ... def close ... def flush ... ``` ``` scala private[sql] class JacksonGenerator( schema: StructType, writer: Writer, options: JSONOptions = new JSONOptions(Map.empty[String, String])) { ... def write ... def close ... def flush ... ``` - This PR also makes the classes put in together in a consistent manner with JSON. - `CsvFileFormat` ``` scala CsvFileFormat CsvOutputWriter ``` - `JsonFileFormat` ``` scala JsonFileFormat JsonOutputWriter ``` ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16496 from HyukjinKwon/SPARK-16101-write.
* [SPARK-19267][SS] Fix a race condition when stopping StateStoreShixiong Zhu2017-01-201-27/+61
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? There is a race condition when stopping StateStore which makes `StateStoreSuite.maintenance` flaky. `StateStore.stop` doesn't wait for the running task to finish, and an out-of-date task may fail `doMaintenance` and cancel the new task. Here is a reproducer: https://github.com/zsxwing/spark/commit/dde1b5b106ba034861cf19e16883cfe181faa6f3 This PR adds MaintenanceTask to eliminate the race condition. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16627 from zsxwing/SPARK-19267.
* [SPARK-18589][SQL] Fix Python UDF accessing attributes from both side of joinDavies Liu2017-01-205-13/+30
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? PythonUDF is unevaluable, which can not be used inside a join condition, currently the optimizer will push a PythonUDF which accessing both side of join into the join condition, then the query will fail to plan. This PR fix this issue by checking the expression is evaluable or not before pushing it into Join. ## How was this patch tested? Add a regression test. Author: Davies Liu <davies@databricks.com> Closes #16581 from davies/pyudf_join.
* [SPARK-19314][SS][CATALYST] Do not allow sort before aggregation in ↵Tathagata Das2017-01-202-3/+8
| | | | | | | | | | | | | | | Structured Streaming plan ## What changes were proposed in this pull request? Sort in a streaming plan should be allowed only after a aggregation in complete mode. Currently it is incorrectly allowed when present anywhere in the plan. It gives unpredictable potentially incorrect results. ## How was this patch tested? New test Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16662 from tdas/SPARK-19314.
* [SPARK-19069][CORE] Expose task 'status' and 'duration' in spark history ↵Parag Chaudhari2017-01-2015-5/+360
| | | | | | | | | | | | | | | | server REST API. ## What changes were proposed in this pull request? Although Spark history server UI shows task ‘status’ and ‘duration’ fields, it does not expose these fields in the REST API response. For the Spark history server API users, it is not possible to determine task status and duration. Spark history server has access to task status and duration from event log, but it is not exposing these in API. This patch is proposed to expose task ‘status’ and ‘duration’ fields in Spark history server REST API. ## How was this patch tested? Modified existing test cases in org.apache.spark.deploy.history.HistoryServerSuite. Author: Parag Chaudhari <paragpc@amazon.com> Closes #16473 from paragpc/expose_task_status.
* [SPARK-19302][DOC][MINOR] Fix the wrong item format in security.mdsarutak2017-01-201-0/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In docs/security.md, there is a description as follows. ``` steps to configure the key-stores and the trust-store for the standalone deployment mode is as follows: * Generate a keys pair for each node * Export the public key of the key pair to a file on each node * Import all exported public keys into a single trust-store ``` According to markdown format, the first item should follow a blank line. ## How was this patch tested? Manually tested. Following captures are rendered web page before and after fix. * before ![before](https://cloud.githubusercontent.com/assets/4736016/22136731/b358115c-df19-11e6-8f6c-2f7b65766265.png) * after ![after](https://cloud.githubusercontent.com/assets/4736016/22136745/c6366ff8-df19-11e6-840d-e7e894218f9c.png) Author: sarutak <sarutak@oss.nttdata.co.jp> Closes #16653 from sarutak/SPARK-19302.
* [SPARK-19271][SQL] Change non-cbo estimation of aggregatewangzhenhua2017-01-197-11/+38
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Change non-cbo estimation behavior of aggregate: - If groupExpression is empty, we can know row count (=1) and the corresponding size; - otherwise, estimation falls back to UnaryNode's computeStats method, which should not propagate rowCount and attributeStats in Statistics because they are not estimated in that method. ## How was this patch tested? Added test case Author: wangzhenhua <wangzhenhua@huawei.com> Closes #16631 from wzhfy/aggNoCbo.
* [SPARK-19292][SQL] filter with partition columns should be case-insensitive ↵Wenchen Fan2017-01-193-2/+25
| | | | | | | | | | | | | | | | | | on Hive tables ## What changes were proposed in this pull request? When we query a table with a filter on partitioned columns, we will push the partition filter to the metastore to get matched partitions directly. In `HiveExternalCatalog.listPartitionsByFilter`, we assume the column names in partition filter are already normalized and we don't need to consider case sensitivity. However, `HiveTableScanExec` doesn't follow this assumption. This PR fixes it. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16647 from cloud-fan/bug.
* [SPARK-17912] [SQL] Refactor code generation to get data for ↵Kazuaki Ishizaki2017-01-192-84/+135
| | | | | | | | | | | | | | | | | ColumnVector/ColumnarBatch ## What changes were proposed in this pull request? This PR refactors the code generation part to get data from `ColumnarVector` and `ColumnarBatch` by using a trait `ColumnarBatchScan` for ease of reuse. This is because this part will be reused by several components (e.g. parquet reader, Dataset.cache, and others) since `ColumnarBatch` will be first citizen. This PR is a part of https://github.com/apache/spark/pull/15219. In advance, this PR makes the code generation for `ColumnarVector` and `ColumnarBatch` reuseable as a trait. In general, this is very useful for other components from the reuseability view, too. ## How was this patch tested? tested existing test suites Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #15467 from kiszk/columnarrefactor.
* [SPARK-19295][SQL] IsolatedClientLoader's downloadVersion should log the ↵Yin Huai2017-01-191-0/+1
| | | | | | | | | | | | | | location of downloaded metastore client jars ## What changes were proposed in this pull request? This will help the users to know the location of those downloaded jars when `spark.sql.hive.metastore.jars` is set to `maven`. ## How was this patch tested? jenkins Author: Yin Huai <yhuai@databricks.com> Closes #16649 from yhuai/SPARK-19295.
* [SPARK-16654][CORE] Add UI coverage for Application Level BlacklistingJosé Hiram Soltren2017-01-1936-235/+950
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Builds on top of work in SPARK-8425 to update Application Level Blacklisting in the scheduler. ## What changes were proposed in this pull request? Adds a UI to these patches by: - defining new listener events for blacklisting and unblacklisting, nodes and executors; - sending said events at the relevant points in BlacklistTracker; - adding JSON (de)serialization code for these events; - augmenting the Executors UI page to show which, and how many, executors are blacklisted; - adding a unit test to make sure events are being fired; - adding HistoryServerSuite coverage to verify that the SHS reads these events correctly. - updates the Executor UI to show Blacklisted/Active/Dead as a tri-state in Executors Status Updates .rat-excludes to pass tests. username squito ## How was this patch tested? ./dev/run-tests testOnly org.apache.spark.util.JsonProtocolSuite testOnly org.apache.spark.scheduler.BlacklistTrackerSuite testOnly org.apache.spark.deploy.history.HistoryServerSuite https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh ![blacklist-20161219](https://cloud.githubusercontent.com/assets/1208477/21335321/9eda320a-c623-11e6-8b8c-9c912a73c276.jpg) Author: José Hiram Soltren <jose@cloudera.com> Closes #16346 from jsoltren/SPARK-16654-submit.
* [SPARK-19059][SQL] Unable to retrieve data from parquet table whose name ↵jayadevanmurali2017-01-192-46/+53
| | | | | | | | | | | | | | | | | | | | startswith underscore ## What changes were proposed in this pull request? The initial shouldFilterOut() method invocation filter the root path name(table name in the intial call) and remove if it contains _. I moved the check one level below, so it first list files/directories in the given root path and then apply filter. (Please fill in changes proposed in this fix) ## How was this patch tested? Added new test case for this scenario (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: jayadevanmurali <jayadevan.m@tcs.com> Author: jayadevan <jayadevan.m@tcs.com> Closes #16635 from jayadevanmurali/branch-0.1-SPARK-19059.
* [SPARK-14272][ML] Add Loglikelihood in GaussianMixtureSummaryZheng RuiFeng2017-01-195-4/+27
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? add loglikelihood in GMM.summary ## How was this patch tested? added tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Author: Ruifeng Zheng <ruifengz@foxmail.com> Closes #12064 from zhengruifeng/gmm_metric.
* [SPARK-19265][SQL] make table relation cache general and does not depend on hiveWenchen Fan2017-01-1920-198/+144
| | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? We have a table relation plan cache in `HiveMetastoreCatalog`, which caches a lot of things: file status, resolved data source, inferred schema, etc. However, it doesn't make sense to limit this cache with hive support, we should move it to SQL core module so that users can use this cache without hive support. It can also reduce the size of `HiveMetastoreCatalog`, so that it's easier to remove it eventually. main changes: 1. move the table relation cache to `SessionCatalog` 2. `SessionCatalog.lookupRelation` will return `SimpleCatalogRelation` and the analyzer will convert it to `LogicalRelation` or `MetastoreRelation` later, then `HiveSessionCatalog` doesn't need to override `lookupRelation` anymore 3. `FindDataSourceTable` will read/write the table relation cache. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #16621 from cloud-fan/plan-cache.
* Update known_translations for contributor namesYin Huai2017-01-181-2/+3
| | | | | | | | | ## What changes were proposed in this pull request? Update known_translations per https://github.com/apache/spark/pull/16423#issuecomment-269739634 Author: Yin Huai <yhuai@databricks.com> Closes #16628 from yhuai/known_translations.
* [SPARK-14975][ML] Fixed GBTClassifier to predict probability per training ↵Ilya Matiach2017-01-185-29/+248
| | | | | | | | | | | | | | | | | | instance and fixed interfaces ## What changes were proposed in this pull request? For all of the classifiers in MLLib we can predict probabilities except for GBTClassifier. Also, all classifiers inherit from ProbabilisticClassifier but GBTClassifier strangely inherits from Predictor, which is a bug. This change corrects the interface and adds the ability for the classifier to give a probabilities vector. ## How was this patch tested? The basic ML tests were run after making the changes. I've marked this as WIP as I need to add more tests. Author: Ilya Matiach <ilmat@microsoft.com> Closes #16441 from imatiach-msft/ilmat/fix-GBT.
* [SPARK-19182][DSTREAM] Optimize the lock in StreamingJobProgressListener to ↵uncleGen2017-01-182-8/+13
| | | | | | | | | | | | | | | | | | not block UI when generating Streaming jobs ## What changes were proposed in this pull request? When DStreamGraph is generating a job, it will hold a lock and block other APIs. Because StreamingJobProgressListener (numInactiveReceivers, streamName(streamId: Int), streamIds) needs to call DStreamGraph's methods to access some information, the UI may hang if generating a job is very slow (e.g., talking to the slow Kafka cluster to fetch metadata). It's better to optimize the locks in DStreamGraph and StreamingJobProgressListener to make the UI not block by job generation. ## How was this patch tested? existing ut cc zsxwing Author: uncleGen <hustyugm@gmail.com> Closes #16601 from uncleGen/SPARK-19182.
* [SPARK-19168][STRUCTURED STREAMING] StateStore should be aborted upon errorLiwei Lin2017-01-183-2/+10
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We should call `StateStore.abort()` when there should be any error before the store is committed. ## How was this patch tested? Manually. Author: Liwei Lin <lwlin7@gmail.com> Closes #16547 from lw-lin/append-filter.
* [SPARK-19113][SS][TESTS] Ignore StreamingQueryException thrown from ↵Shixiong Zhu2017-01-181-1/+6
| | | | | | | | | | | | | | | | awaitInitialization to avoid breaking tests ## What changes were proposed in this pull request? #16492 missed one race condition: `StreamExecution.awaitInitialization` may throw fatal errors and fail the test. This PR just ignores `StreamingQueryException` thrown from `awaitInitialization` so that we can verify the exception in the `ExpectFailure` action later. It's fine since `StopStream` or `ExpectFailure` will catch `StreamingQueryException` as well. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16567 from zsxwing/SPARK-19113-2.
* [SPARK-18113] Use ask to replace askWithRetry in canCommit and make receiver ↵jinxing2017-01-182-4/+31
| | | | | | | | | | | | | | | | | | | | idempotent. ## What changes were proposed in this pull request? Method canCommit sends AskPermissionToCommitOutput using askWithRetry. If timeout, it will send again. Thus AskPermissionToCommitOutput can be received multi times. Method canCommit should return the same value when called by the same attempt multi times. In implementation before this fix, method handleAskPermissionToCommit just check if there is committer already registered, which is not enough. When worker retries AskPermissionToCommitOutput it will get CommitDeniedException, then the task will fail with reason TaskCommitDenied, which is not regarded as a task failure(SPARK-11178), so TaskScheduler will schedule this task infinitely. In this fix, use `ask` to replace `askWithRetry` in `canCommit` and make receiver idempotent. ## How was this patch tested? Added a new unit test to OutputCommitCoordinatorSuite. Author: jinxing <jinxing@meituan.com> Closes #16503 from jinxing64/SPARK-18113.
* [SPARK-19231][SPARKR] add error handling for download and untar for Spark ↵Felix Cheung2017-01-181-15/+40
| | | | | | | | | | | | | | | | release ## What changes were proposed in this pull request? When R is starting as a package and it needs to download the Spark release distribution we need to handle error for download and untar, and clean up, otherwise it will get stuck. ## How was this patch tested? manually Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16589 from felixcheung/rtarreturncode.
* [SPARK-19223][SQL][PYSPARK] Fix InputFileBlockHolder for datasources which ↵Liang-Chi Hsieh2017-01-182-3/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | are based on HadoopRDD or NewHadoopRDD ## What changes were proposed in this pull request? For some datasources which are based on HadoopRDD or NewHadoopRDD, such as spark-xml, InputFileBlockHolder doesn't work with Python UDF. The method to reproduce it is, running the following codes with `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`: from pyspark.sql.functions import udf,input_file_name from pyspark.sql.types import StringType from pyspark.sql import SparkSession def filename(path): return path session = SparkSession.builder.appName('APP').getOrCreate() session.udf.register('sameText', filename) sameText = udf(filename, StringType()) df = session.read.format('xml').load('a.xml', rowTag='root').select('*', input_file_name().alias('file')) df.select('file').show() # works df.select(sameText(df['file'])).show() # returns empty content The issue is because in `HadoopRDD` and `NewHadoopRDD` we set the file block's info in `InputFileBlockHolder` before the returned iterator begins consuming. `InputFileBlockHolder` will record this info into thread local variable. When running Python UDF in batch, we set up another thread to consume the iterator from child plan's output rdd, so we can't read the info back in another thread. To fix this, we have to set the info in `InputFileBlockHolder` after the iterator begins consuming. So the info can be read in correct thread. ## How was this patch tested? Manual test with above example codes for spark-xml package on pyspark: `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`. Added pyspark test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16585 from viirya/fix-inputfileblock-hadooprdd.
* [SPARK-19024][SQL] Implement new approach to write a permanent viewjiangxingbo2017-01-185-92/+146
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? On CREATE/ALTER a view, it's no longer needed to generate a SQL text string from the LogicalPlan, instead we store the SQL query text、the output column names of the query plan, and current database to CatalogTable. Permanent views created by this approach can be resolved by current view resolution approach. The main advantage includes: 1. If you update an underlying view, the current view also gets updated; 2. That gives us a change to get ride of SQL generation for operators. Major changes of this PR: 1. Generate the view-specific properties(e.g. view default database, view query output column names) during permanent view creation and store them as properties in the CatalogTable; 2. Update the commands `CreateViewCommand` and `AlterViewAsCommand`, get rid of SQL generation from them. ## How was this patch tested? Existing tests. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #16613 from jiangxb1987/view-write-path.
* [SPARK-18782][BUILD] Bump Hadoop 2.6 version to use Hadoop 2.6.5Adam Roberts2017-01-182-16/+16
| | | | | | | | | | | | | | **What changes were proposed in this pull request?** Use Hadoop 2.6.5 for the Hadoop 2.6 profile, I see a bunch of fixes including security ones in the release notes that we should pick up **How was this patch tested?** Running the unit tests now with IBM's SDK for Java and let's see what happens with OpenJDK in the community builder - expecting no trouble as it is only a minor release. Author: Adam Roberts <aroberts@uk.ibm.com> Closes #16616 from a-roberts/Hadoop265Bumper.
* [SPARK-19227][SPARK-19251] remove unused imports and outdated commentsuncleGen2017-01-1847-79/+25
| | | | | | | | | | | | ## What changes were proposed in this pull request? remove ununsed imports and outdated comments, and fix some minor code style issue. ## How was this patch tested? existing ut Author: uncleGen <hustyugm@gmail.com> Closes #16591 from uncleGen/SPARK-19227.
* [SPARK-18243][SQL] Port Hive writing to use FileFormat interfaceWenchen Fan2017-01-1715-533/+318
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Inserting data into Hive tables has its own implementation that is distinct from data sources: `InsertIntoHiveTable`, `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`. Note that one other major difference is that data source tables write directly to the final destination without using some staging directory, and then Spark itself adds the partitions/tables to the catalog. Hive tables actually write to some staging directory, and then call Hive metastore's loadPartition/loadTable function to load those data in. So we still need to keep `InsertIntoHiveTable` to put this special logic. In the future, we should think of writing to the hive table location directly, so that we don't need to call `loadTable`/`loadPartition` at the end and remove `InsertIntoHiveTable`. This PR removes `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`, and create a `HiveFileFormat` to implement the write logic. In the future, we should also implement the read logic in `HiveFileFormat`. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #16517 from cloud-fan/insert-hive.
* [SPARK-18206][ML] Add instrumentation for MLP,NB,LDA,AFT,GLM,Isotonic,LiRZheng RuiFeng2017-01-1713-33/+105
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? add instrumentation for MLP,NB,LDA,AFT,GLM,Isotonic,LiR ## How was this patch tested? local test in spark-shell Author: Zheng RuiFeng <ruifengz@foxmail.com> Author: Ruifeng Zheng <ruifengz@foxmail.com> Closes #15671 from zhengruifeng/lir_instr.
* [SPARK-13721][SQL] Support outer generators in DataFrame APIBogdan Raducanu2017-01-179-16/+150
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Added outer_explode, outer_posexplode, outer_inline functions and expressions. Some bug fixing in GenerateExec.scala for CollectionGenerator. Previously it was not correctly handling the case of outer with empty collections, only with nulls. ## How was this patch tested? New tests added to GeneratorFunctionSuite Author: Bogdan Raducanu <bogdan.rdc@gmail.com> Closes #16608 from bogdanrdc/SPARK-13721.
* [SPARK-18917][SQL] Remove schema check in appending dataReynold Xin2017-01-171-33/+3
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In append mode, we check whether the schema of the write is compatible with the schema of the existing data. It can be a significant performance issue in cloud environment to find the existing schema for files. This patch removes the check. Note that for catalog tables, we always do the check, as discussed in https://github.com/apache/spark/pull/16339#discussion_r96208357 ## How was this patch tested? N/A Closes #16339. Author: Reynold Xin <rxin@databricks.com> Closes #16622 from rxin/SPARK-18917.
* [MINOR][SQL] Remove duplicate call of reset() function in ↵jiangxingbo2017-01-171-1/+0
| | | | | | | | | | | | | | | | CurrentOrigin.withOrigin() ## What changes were proposed in this pull request? Remove duplicate call of reset() function in CurrentOrigin.withOrigin(). ## How was this patch tested? Existing test cases. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #16615 from jiangxb1987/dummy-code.
* [SPARK-19239][PYSPARK] Check parameters whether equals None when specify the ↵DjvuLee2017-01-171-3/+6
| | | | | | | | | | | | | | | | | | | | column in jdbc API ## What changes were proposed in this pull request? The `jdbc` API do not check the `lowerBound` and `upperBound` when we specified the ``column``, and just throw the following exception: >```int() argument must be a string or a number, not 'NoneType'``` If we check the parameter, we can give a more friendly suggestion. ## How was this patch tested? Test using the pyspark shell, without the lowerBound and upperBound parameters. Author: DjvuLee <lihu@bytedance.com> Closes #16599 from djvulee/pysparkFix.
* [SPARK-19129][SQL] SessionCatalog: Disallow empty part col values in ↵gatorsmile2017-01-185-8/+106
| | | | | | | | | | | | | | | | | | | | | | | | | partition spec ### What changes were proposed in this pull request? Empty partition column values are not valid for partition specification. Before this PR, we accept users to do it; however, Hive metastore does not detect and disallow it too. Thus, users hit the following strange error. ```Scala val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name") df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable") spark.sql("alter table partitionedTable drop partition(partCol1='')") spark.table("partitionedTable").show() ``` In the above example, the WHOLE table is DROPPED when users specify a partition spec containing only one partition column with empty values. When the partition columns contains more than one, Hive metastore APIs simply ignore the columns with empty values and treat it as partial spec. This is also not expected. This does not follow the actual Hive behaviors. This PR is to disallow users to specify such an invalid partition spec in the `SessionCatalog` APIs. ### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #16583 from gatorsmile/disallowEmptyPartColValue.
* [SPARK-19065][SQL] Don't inherit expression id in dropDuplicatesShixiong Zhu2017-01-183-11/+27
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `dropDuplicates` will create an Alias using the same exprId, so `StreamExecution` should also replace Alias if necessary. ## How was this patch tested? test("SPARK-19065: dropDuplicates should not create expressions using the same id") Author: Shixiong Zhu <shixiong@databricks.com> Closes #16564 from zsxwing/SPARK-19065.
* [SPARK-19019] [PYTHON] Fix hijacked `collections.namedtuple` and port ↵hyukjinkwon2017-01-172-31/+87
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | cloudpickle changes for PySpark to work with Python 3.6.0 ## What changes were proposed in this pull request? Currently, PySpark does not work with Python 3.6.0. Running `./bin/pyspark` simply throws the error as below and PySpark does not work at all: ``` Traceback (most recent call last): File ".../spark/python/pyspark/shell.py", line 30, in <module> import pyspark File ".../spark/python/pyspark/__init__.py", line 46, in <module> from pyspark.context import SparkContext File ".../spark/python/pyspark/context.py", line 36, in <module> from pyspark.java_gateway import launch_gateway File ".../spark/python/pyspark/java_gateway.py", line 31, in <module> from py4j.java_gateway import java_import, JavaGateway, GatewayClient File "<frozen importlib._bootstrap>", line 961, in _find_and_load File "<frozen importlib._bootstrap>", line 950, in _find_and_load_unlocked File "<frozen importlib._bootstrap>", line 646, in _load_unlocked File "<frozen importlib._bootstrap>", line 616, in _load_backward_compatible File ".../spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 18, in <module> File "/usr/local/Cellar/python3/3.6.0/Frameworks/Python.framework/Versions/3.6/lib/python3.6/pydoc.py", line 62, in <module> import pkgutil File "/usr/local/Cellar/python3/3.6.0/Frameworks/Python.framework/Versions/3.6/lib/python3.6/pkgutil.py", line 22, in <module> ModuleInfo = namedtuple('ModuleInfo', 'module_finder name ispkg') File ".../spark/python/pyspark/serializers.py", line 394, in namedtuple cls = _old_namedtuple(*args, **kwargs) TypeError: namedtuple() missing 3 required keyword-only arguments: 'verbose', 'rename', and 'module' ``` The root cause seems because some arguments of `namedtuple` are now completely keyword-only arguments from Python 3.6.0 (See https://bugs.python.org/issue25628). We currently copy this function via `types.FunctionType` which does not set the default values of keyword-only arguments (meaning `namedtuple.__kwdefaults__`) and this seems causing internally missing values in the function (non-bound arguments). This PR proposes to work around this by manually setting it via `kwargs` as `types.FunctionType` seems not supporting to set this. Also, this PR ports the changes in cloudpickle for compatibility for Python 3.6.0. ## How was this patch tested? Manually tested with Python 2.7.6 and Python 3.6.0. ``` ./bin/pyspsark ``` , manual creation of `namedtuple` both in local and rdd with Python 3.6.0, and Jenkins tests for other Python versions. Also, ``` ./run-tests --python-executables=python3.6 ``` ``` Will test against the following Python executables: ['python3.6'] Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming'] Finished test(python3.6): pyspark.sql.tests (192s) Finished test(python3.6): pyspark.accumulators (3s) Finished test(python3.6): pyspark.mllib.tests (198s) Finished test(python3.6): pyspark.broadcast (3s) Finished test(python3.6): pyspark.conf (2s) Finished test(python3.6): pyspark.context (14s) Finished test(python3.6): pyspark.ml.classification (21s) Finished test(python3.6): pyspark.ml.evaluation (11s) Finished test(python3.6): pyspark.ml.clustering (20s) Finished test(python3.6): pyspark.ml.linalg.__init__ (0s) Finished test(python3.6): pyspark.streaming.tests (240s) Finished test(python3.6): pyspark.tests (240s) Finished test(python3.6): pyspark.ml.recommendation (19s) Finished test(python3.6): pyspark.ml.feature (36s) Finished test(python3.6): pyspark.ml.regression (37s) Finished test(python3.6): pyspark.ml.tuning (28s) Finished test(python3.6): pyspark.mllib.classification (26s) Finished test(python3.6): pyspark.mllib.evaluation (18s) Finished test(python3.6): pyspark.mllib.clustering (44s) Finished test(python3.6): pyspark.mllib.linalg.__init__ (0s) Finished test(python3.6): pyspark.mllib.feature (26s) Finished test(python3.6): pyspark.mllib.fpm (23s) Finished test(python3.6): pyspark.mllib.random (8s) Finished test(python3.6): pyspark.ml.tests (92s) Finished test(python3.6): pyspark.mllib.stat.KernelDensity (0s) Finished test(python3.6): pyspark.mllib.linalg.distributed (25s) Finished test(python3.6): pyspark.mllib.stat._statistics (15s) Finished test(python3.6): pyspark.mllib.recommendation (24s) Finished test(python3.6): pyspark.mllib.regression (26s) Finished test(python3.6): pyspark.profiler (9s) Finished test(python3.6): pyspark.mllib.tree (16s) Finished test(python3.6): pyspark.shuffle (1s) Finished test(python3.6): pyspark.mllib.util (18s) Finished test(python3.6): pyspark.serializers (11s) Finished test(python3.6): pyspark.rdd (20s) Finished test(python3.6): pyspark.sql.conf (8s) Finished test(python3.6): pyspark.sql.catalog (17s) Finished test(python3.6): pyspark.sql.column (18s) Finished test(python3.6): pyspark.sql.context (18s) Finished test(python3.6): pyspark.sql.group (27s) Finished test(python3.6): pyspark.sql.dataframe (33s) Finished test(python3.6): pyspark.sql.functions (35s) Finished test(python3.6): pyspark.sql.types (6s) Finished test(python3.6): pyspark.sql.streaming (13s) Finished test(python3.6): pyspark.streaming.util (0s) Finished test(python3.6): pyspark.sql.session (16s) Finished test(python3.6): pyspark.sql.window (4s) Finished test(python3.6): pyspark.sql.readwriter (35s) Tests passed in 433 seconds ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16429 from HyukjinKwon/SPARK-19019.
* [SPARK-19179][YARN] Change spark.yarn.access.namenodes config and update docsjerryshao2017-01-174-15/+23
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `spark.yarn.access.namenodes` configuration cannot actually reflects the usage of it, inside the code it is the Hadoop filesystems we get tokens, not NNs. So here propose to update the name of this configuration, also change the related code and doc. ## How was this patch tested? Local verification. Author: jerryshao <sshao@hortonworks.com> Closes #16560 from jerryshao/SPARK-19179.
* [SPARK-3249][DOC] Fix links in ScalaDoc that cause warning messages in ↵hyukjinkwon2017-01-1720-86/+93
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | `sbt/sbt unidoc` ## What changes were proposed in this pull request? This PR proposes to fix ambiguous link warnings by simply making them as code blocks for both javadoc and scaladoc. ``` [warn] .../spark/core/src/main/scala/org/apache/spark/Accumulator.scala:20: The link target "SparkContext#accumulator" is ambiguous. Several members fit the target: [warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala:281: The link target "runMiniBatchSGD" is ambiguous. Several members fit the target: [warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala:83: The link target "run" is ambiguous. Several members fit the target: ... ``` This PR also fixes javadoc8 break as below: ``` [error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found [error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product} [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found [error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product} [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found [error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product} [error] ^ [info] 3 errors ``` ## How was this patch tested? Manually via `sbt unidoc > output.txt` and the checked it via `cat output.txt | grep ambiguous` and `sbt unidoc | grep error`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16604 from HyukjinKwon/SPARK-3249.
* [SPARK-19219][SQL] Fix Parquet log output defaultsNick Lavers2017-01-173-4/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Changing the default parquet logging levels to reflect the changes made in PR [#15538](https://github.com/apache/spark/pull/15538), in order to prevent the flood of log messages by default. ## How was this patch tested? Default log output when reading from parquet 1.6 files was compared with and without this change. The change eliminates the extraneous logging and makes the output readable. Author: Nick Lavers <nick.lavers@videoamp.com> Closes #16580 from nicklavers/spark-19219-set_default_parquet_log_level.
* [SPARK-19240][SQL][TEST] add test for setting location for managed tableWenchen Fan2017-01-171-0/+28
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? SET LOCATION can also work on managed table(or table created without custom path), the behavior is a little weird, but as we have already supported it, we should add a test to explicitly show the behavior. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #16597 from cloud-fan/set-location.
* [MINOR][YARN] Move YarnSchedulerBackendSuite to resource-managers/yarn ↵Yanbo Liang2017-01-171-0/+0
| | | | | | | | | | | | | | directory. ## What changes were proposed in this pull request? #16092 moves YARN resource manager related code to resource-managers/yarn directory. The test case ```YarnSchedulerBackendSuite``` was added after that but with the wrong place. I move it to correct directory in this PR. ## How was this patch tested? Existing test. Author: Yanbo Liang <ybliang8@gmail.com> Closes #16595 from yanboliang/yarn.