aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-15037] [SQL] [MLLIB] Part2: Use SparkSession instead of SQLContext in ↵Sandeep Singh2016-05-114-294/+273
| | | | | | | | | | | | | | Python TestSuites ## What changes were proposed in this pull request? Use SparkSession instead of SQLContext in Python TestSuites ## How was this patch tested? Existing tests Author: Sandeep Singh <sandeep@techaddict.me> Closes #13044 from techaddict/SPARK-15037-python.
* [SPARK-15241] [SPARK-15242] [SQL] fix 2 decimal-related issues in RowEncoderWenchen Fan2016-05-114-10/+29
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-15241: We now support java decimal and catalyst decimal in external row, it makes sense to also support scala decimal. SPARK-15242: This is a long-standing bug, and is exposed after https://github.com/apache/spark/pull/12364, which eliminate the `If` expression if the field is not nullable: ``` val fieldValue = serializerFor( GetExternalRowField(inputObject, i, externalDataTypeForInput(f.dataType)), f.dataType) if (f.nullable) { If( Invoke(inputObject, "isNullAt", BooleanType, Literal(i) :: Nil), Literal.create(null, f.dataType), fieldValue) } else { fieldValue } ``` Previously, we always use `DecimalType.SYSTEM_DEFAULT` as the output type of converted decimal field, which is wrong as it doesn't match the real decimal type. However, it works well because we always put converted field into `If` expression to do the null check, and `If` use its `trueValue`'s data type as its output type. Now if we have a not nullable decimal field, then the converted field's output type will be `DecimalType.SYSTEM_DEFAULT`, and we will write wrong data into unsafe row. The fix is simple, just use the given decimal type as the output type of converted decimal field. These 2 issues was found at https://github.com/apache/spark/pull/13008 ## How was this patch tested? new tests in RowEncoderSuite Author: Wenchen Fan <wenchen@databricks.com> Closes #13019 from cloud-fan/encoder-decimal.
* [SPARK-14933][HOTFIX] Replace `sqlContext` with `spark`.Dongjoon Hyun2016-05-112-5/+5
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This fixes compile errors. ## How was this patch tested? Pass the Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13053 from dongjoon-hyun/hotfix_sqlquerysuite.
* [SPARK-15268][SQL] Make JavaTypeInference work with UDTRegistrationLiang-Chi Hsieh2016-05-112-0/+21
| | | | | | | | | | | | | ## What changes were proposed in this pull request? We have a private `UDTRegistration` API to register user defined type. Currently `JavaTypeInference` can't work with it. So `SparkSession.createDataFrame` from a bean class will not correctly infer the schema of the bean class. ## How was this patch tested? `VectorUDTSuite`. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #13046 from viirya/fix-udt-registry-javatypeinference.
* [SPARK-14933][SQL] Failed to create view out of a parquet or orc tablexin Wu2016-05-113-2/+62
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? #### Symptom If a table is created as parquet or ORC table with hive syntaxt DDL, such as ```SQL create table t1 (c1 int, c2 string) stored as parquet ``` The following command will fail ```SQL create view v1 as select * from t1 ``` #### Root Cause Currently, `HiveMetaStoreCatalog` converts Paruqet/Orc tables to `LogicalRelation` without giving any `tableIdentifier`. `SQLBuilder` expects the `LogicalRelation` to have an associated `tableIdentifier`. However, the `LogicalRelation` created earlier does not have such a `tableIdentifier`. Thus, `SQLBuilder.toSQL` can not recognize this logical plan and issue an exception. This PR is to assign a `TableIdentifier` to the `LogicalRelation` when resolving parquet or orc tables in `HiveMetaStoreCatalog`. ## How was this patch tested? testcases created and dev/run-tests is run. Author: xin Wu <xinwu@us.ibm.com> Closes #12716 from xwu0226/SPARK_14933.
* [SPARK-15150][EXAMPLE][DOC] Update LDA examplesZheng RuiFeng2016-05-118-80/+143
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1,create a libsvm-type dataset for lda: `data/mllib/sample_lda_libsvm_data.txt` 2,add python example 3,directly read the datafile in examples 4,BTW, change to `SparkSession` in `aft_survival_regression.py` ## How was this patch tested? manual tests `./bin/spark-submit examples/src/main/python/ml/lda_example.py` Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #12927 from zhengruifeng/lda_pe.
* [SPARK-15238] Clarify supported Python versionsNicholas Chammas2016-05-111-2/+2
| | | | | | | | | This PR: * Clarifies that Spark *does* support Python 3, starting with Python 3.4. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #13017 from nchammas/supported-python-versions.
* [SPARK-14976][STREAMING] make StreamingContext.textFileStream support wildcardmwws2016-05-112-2/+70
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? make StreamingContext.textFileStream support wildcard like /home/user/*/file ## How was this patch tested? I did manual test and added a new unit test case Author: mwws <wei.mao@intel.com> Author: unknown <maowei@maowei-MOBL.ccr.corp.intel.com> Closes #12752 from mwws/SPARK_FileStream.
* [SPARK-15149][EXAMPLE][DOC] update kmeans exampleZheng RuiFeng2016-05-114-94/+50
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Python example for ml.kmeans already exists, but not included in user guide. 1,small changes like: `example_on` `example_off` 2,add it to user guide 3,update examples to directly read datafile ## How was this patch tested? manual tests `./bin/spark-submit examples/src/main/python/ml/kmeans_example.py Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #12925 from zhengruifeng/km_pe.
* [SPARK-14340][EXAMPLE][DOC] Update Examples and User Guide for ↵Zheng RuiFeng2016-05-115-48/+139
| | | | | | | | | | | | | | | | | | ml.BisectingKMeans ## What changes were proposed in this pull request? 1, add BisectingKMeans to ml-clustering.md 2, add the missing Scala BisectingKMeansExample 3, create a new datafile `data/mllib/sample_kmeans_data.txt` ## How was this patch tested? manual tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #11844 from zhengruifeng/doc_bkm.
* [SPARK-15141][EXAMPLE][DOC] Update OneVsRest ExamplesZheng RuiFeng2016-05-114-316/+129
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1, Add python example for OneVsRest 2, remove args-parsing ## How was this patch tested? manual tests `./bin/spark-submit examples/src/main/python/ml/one_vs_rest_example.py` Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #12920 from zhengruifeng/ovr_pe.
* [SPARK-15231][SQL] Document the semantic of saveAsTable and insertInto and ↵Shixiong Zhu2016-05-103-2/+82
| | | | | | | | | | | | | | | | don't drop columns silently ## What changes were proposed in this pull request? This PR adds documents about the different behaviors between `insertInto` and `saveAsTable`, and throws an exception when the user try to add too man columns using `saveAsTable with append`. ## How was this patch tested? Unit tests added in this PR. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13013 from zsxwing/SPARK-15231.
* [SPARK-15189][PYSPARK][DOCS] Update ml.evaluation PyDocHolden Karau2016-05-111-1/+12
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix doctest issue, short param description, and tag items as Experimental ## How was this patch tested? build docs locally & doctests Author: Holden Karau <holden@us.ibm.com> Closes #12964 from holdenk/SPARK-15189-ml.Evaluation-PyDoc-issues.
* [SPARK-15235][WEBUI] Corresponding row cannot be highlighted even though ↵Kousuke Saruta2016-05-101-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | cursor is on the job on Web UI's timeline ## What changes were proposed in this pull request? To extract job descriptions and stage name, there are following regular expressions in timeline-view.js ``` var jobIdText = $($(baseElem).find(".application-timeline-content")[0]).text(); var jobId = jobIdText.match("\\(Job (\\d+)\\)")[1]; ... var stageIdText = $($(baseElem).find(".job-timeline-content")[0]).text(); var stageIdAndAttempt = stageIdText.match("\\(Stage (\\d+\\.\\d+)\\)")[1].split("."); ``` But if job descriptions include patterns like "(Job x)" or stage names include patterns like "(Stage x.y)", the regular expressions cannot be match as we expected, ending up with corresponding row cannot be highlighted even though we move the cursor onto the job on Web UI's timeline. ## How was this patch tested? Manually tested with spark-shell and Web UI. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #13016 from sarutak/SPARK-15235.
* [SPARK-15246][SPARK-4452][CORE] Fix code style and improve volatile forLianhui Wang2016-05-102-3/+2
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1. Fix code style 2. remove volatile of elementsRead method because there is only one thread to use it. 3. avoid volatile of _elementsRead because Collection increases number of _elementsRead when it insert a element. It is very expensive. So we can avoid it. After this PR, I will push another PR for branch 1.6. ## How was this patch tested? unit tests Author: Lianhui Wang <lianhuiwang09@gmail.com> Closes #13020 from lianhuiwang/SPARK-4452-hotfix.
* [SPARK-15255][SQL] limit the length of name for cached DataFrameDavies Liu2016-05-101-1/+5
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? We use the tree string of an SparkPlan as the name of cached DataFrame, that could be very long, cause the browser to be not responsive. This PR will limit the length of the name to 1000 characters. ## How was this patch tested? Here is how the UI looks right now: ![ui](https://cloud.githubusercontent.com/assets/40902/15163355/d5640f9c-16bc-11e6-8655-809af8a4fed1.png) Author: Davies Liu <davies@databricks.com> Closes #13033 from davies/cache_name.
* [SPARK-15265][SQL][MINOR] Fix Union query error message indentationDongjoon Hyun2016-05-101-6/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This issue fixes the error message indentation consistently with other set queries (EXCEPT/INTERSECT). **Before (4 lines)** ``` scala> sql("(select 1) union (select 1, 2)").head org.apache.spark.sql.AnalysisException: Unions can only be performed on tables with the same number of columns, but one table has '2' columns and another table has '1' columns; ``` **After (one-line)** ``` scala> sql("(select 1) union (select 1, 2)").head org.apache.spark.sql.AnalysisException: Unions can only be performed on tables with the same number of columns, but one table has '2' columns and another table has '1' columns; ``` **Reference (EXCEPT / INTERSECT)** ``` scala> sql("(select 1) intersect (select 1, 2)").head org.apache.spark.sql.AnalysisException: Intersect can only be performed on tables with the same number of columns, but the left table has 1 columns and the right has 2; ``` ## How was this patch tested? Manual. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13043 from dongjoon-hyun/SPARK-15265.
* [SPARK-15250][SQL] Remove deprecated json API in DataFrameReaderhyukjinkwon2016-05-103-34/+6
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR removes the old `json(path: String)` API which is covered by the new `json(paths: String*)`. ## How was this patch tested? Jenkins tests (existing tests should cover this) Author: hyukjinkwon <gurwls223@gmail.com> Author: Hyukjin Kwon <gurwls223@gmail.com> Closes #13040 from HyukjinKwon/SPARK-15250.
* [SPARK-15261][SQL] Remove experimental tag from DataFrameReader/WriterReynold Xin2016-05-103-13/+23
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch removes experimental tag from DataFrameReader and DataFrameWriter, and explicitly tags a few methods added for structured streaming as experimental. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #13038 from rxin/SPARK-15261.
* [SPARK-14476][SQL] Improve the physical plan visualization by adding meta ↵Sean Zhong2016-05-105-27/+53
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | info like table name and file path for data source. ## What changes were proposed in this pull request? Improve the physical plan visualization by adding meta info like table name and file path for data source. Meta info InputPaths and TableName are newly added. Example: ``` scala> spark.range(10).write.saveAsTable("tt") scala> spark.sql("select * from tt").explain() == Physical Plan == WholeStageCodegen : +- BatchedScan HadoopFiles[id#13L] Format: ParquetFormat, InputPaths: file:/home/xzhong10/spark-linux/assembly/spark-warehouse/tt, PushedFilters: [], ReadSchema: struct<id:bigint>, TableName: default.tt ``` ## How was this patch tested? manual tests. Changes for UI: Before: ![ui_before_change](https://cloud.githubusercontent.com/assets/2595532/15064559/3d423e3c-1388-11e6-8099-7803ef496c4d.jpg) After: ![fix_long_string](https://cloud.githubusercontent.com/assets/2595532/15133566/8ad09e26-1696-11e6-939c-99b908249b9d.jpg) ![for_load](https://cloud.githubusercontent.com/assets/2595532/15157224/3ba95c98-171d-11e6-885a-de0ee8dec27c.jpg) Author: Sean Zhong <clockfly@gmail.com> Closes #12947 from clockfly/spark-14476.
* [SPARK-14837][SQL][STREAMING] Added support in file stream source for ↵Tathagata Das2016-05-105-37/+114
| | | | | | | | | | | | | | | reading new files added to subdirs ## What changes were proposed in this pull request? Currently, file stream source can only find new files if they appear in the directory given to the source, but not if they appear in subdirs. This PR add support for providing glob patterns when creating file stream source so that it can find new files in nested directories based on the glob pattern. ## How was this patch tested? Unit test that tests when new files are discovered with globs and partitioned directories. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #12616 from tdas/SPARK-14837.
* [SPARK-14936][BUILD][TESTS] FlumePollingStreamSuite is slowXin Ren2016-05-103-9/+23
| | | | | | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-14936 ## What changes were proposed in this pull request? FlumePollingStreamSuite contains two tests which run for a minute each. This seems excessively slow and we should speed it up if possible. In this PR, instead of creating `StreamingContext` directly from `conf`, here an underlying `SparkContext` is created before all and it is used to create each`StreamingContext`. Running time is reduced by avoiding multiple `SparkContext` creations and destroys. ## How was this patch tested? Tested on my local machine running `testOnly *.FlumePollingStreamSuite` Author: Xin Ren <iamshrek@126.com> Closes #12845 from keypointt/SPARK-14936.
* [SPARK-15249][SQL] Use FunctionResource instead of (String, String) in ↵Sandeep Singh2016-05-1010-27/+27
| | | | | | | | | | | | | | | CreateFunction and CatalogFunction for resource Use FunctionResource instead of (String, String) in CreateFunction and CatalogFunction for resource see: TODO's here https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala#L36 https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala#L42 Existing tests Author: Sandeep Singh <sandeep@techaddict.me> Closes #13024 from techaddict/SPARK-15249.
* [SPARK-6005][TESTS] Fix flaky test: ↵Shixiong Zhu2016-05-101-6/+14
| | | | | | | | | | | | | | | | | | o.a.s.streaming.kafka.DirectKafkaStreamSuite.offset recovery ## What changes were proposed in this pull request? Because this test extracts data from `DStream.generatedRDDs` before stopping, it may get data before checkpointing. Then after recovering from the checkpoint, `recoveredOffsetRanges` may contain something not in `offsetRangesBeforeStop`, which will fail the test. Adding `Thread.sleep(1000)` before `ssc.stop()` will reproduce this failure. This PR just moves the logic of `offsetRangesBeforeStop` (also renamed to `offsetRangesAfterStop`) after `ssc.stop()` to fix the flaky test. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #12903 from zsxwing/SPARK-6005.
* [SPARK-15207][BUILD] Use Travis CI for Java Linter and JDK7/8 compilation testDongjoon Hyun2016-05-101-0/+51
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, Java Linter is disabled in Jenkins tests. https://github.com/apache/spark/blob/master/dev/run-tests.py#L554 However, as of today, Spark has 721 java files with 97362 code (without blank/comments). It's about 1/3 of Scala. ``` -------------------------------------------------------------------------------- Language files blank comment code -------------------------------------------------------------------------------- Scala 2353 62819 124060 318747 Java 721 18617 23314 97362 ``` This PR aims to take advantage of Travis CI to handle the following static analysis by adding a single file, `.travis.yml` without any additional burden on the existing servers. - Java Linter - JDK7/JDK8 maven compile Note that this PR does not propose to remove some of the above work items from the Jenkins. It's possible, but we need to observe the Travis CI stability for a while. The main goal of this issue is to remove committer's overhead on linter-related PRs (the original PR and the fixation PR). ## How was this patch tested? Pass the Travis CI tests. Please see the following link. https://travis-ci.org/dongjoon-hyun/spark/builds/128595350 https://travis-ci.org/dongjoon-hyun/spark/builds/128708372 Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12980 from dongjoon-hyun/SPARK-15207.
* [SPARK-14986][SQL] Return correct result for empty LATERAL VIEW OUTERHerman van Hovell2016-05-102-1/+9
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? A Generate with the `outer` flag enabled should always return one or more rows for every input row. The optimizer currently violates this by rewriting `outer` Generates that do not contain columns of the child plan into an unjoined generate, for example: ```sql select e from a lateral view outer explode(a.b) as e ``` The result of this is that `outer` Generate does not produce output at all when the Generators' input expression is empty. This PR fixes this. ## How was this patch tested? Added test case to `SQLQuerySuite`. Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #12906 from hvanhovell/SPARK-14986.
* [SPARK-14642][SQL] import org.apache.spark.sql.expressions._ breaks udf ↵Subhobrata Dey2016-05-108-10/+10
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | under functions ## What changes were proposed in this pull request? PR fixes the import issue which breaks udf functions. The following code snippet throws an error ``` scala> import org.apache.spark.sql.functions._ import org.apache.spark.sql.functions._ scala> import org.apache.spark.sql.expressions._ import org.apache.spark.sql.expressions._ scala> udf((v: String) => v.stripSuffix("-abc")) <console>:30: error: No TypeTag available for String udf((v: String) => v.stripSuffix("-abc")) ``` This PR resolves the issue. ## How was this patch tested? patch tested with unit tests. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Subhobrata Dey <sbcd90@gmail.com> Closes #12458 from sbcd90/udfFuncBreak.
* [SPARK-15195][PYSPARK][DOCS] Update ml.tuning PyDocsHolden Karau2016-05-101-1/+15
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Tag classes in ml.tuning as experimental, add docs for kfolds avg metric, and copy TrainValidationSplit scaladoc for more detailed explanation. ## How was this patch tested? built docs locally Author: Holden Karau <holden@us.ibm.com> Closes #12967 from holdenk/SPARK-15195-pydoc-ml-tuning.
* [SPARK-15037][HOTFIX] Don't create 2 SparkSessions in constructorAndrew Or2016-05-101-11/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? After #12907 `TestSparkSession` creates a spark session in one of the constructors just to get the `SparkContext` from it. This ends up creating 2 `SparkSession`s from one call, which is definitely not what we want. ## How was this patch tested? Jenkins. Author: Andrew Or <andrew@databricks.com> Closes #13031 from andrewor14/sql-test.
* [SPARK-15037][HOTFIX] Replace `sqlContext` and `sparkSession` with `spark`.Dongjoon Hyun2016-05-101-2/+2
| | | | | | | | | | This replaces `sparkSession` with `spark` in CatalogSuite.scala. Pass the Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13030 from dongjoon-hyun/hotfix_sparkSession.
* [HOTFIX] SQL test compilation error from merge conflictAndrew Or2016-05-101-4/+4
|
* [SPARK-14603][SQL] Verification of Metadata Operations by Session Cataloggatorsmile2016-05-1010-123/+261
| | | | | | | | | | | | | | | | | | Since we cannot really trust if the underlying external catalog can throw exceptions when there is an invalid metadata operation, let's do it in SessionCatalog. - [X] The first step is to unify the error messages issued in Hive-specific Session Catalog and general Session Catalog. - [X] The second step is to verify the inputs of metadata operations for partitioning-related operations. This is moved to a separate PR: https://github.com/apache/spark/pull/12801 - [X] The third step is to add database existence verification in `SessionCatalog` - [X] The fourth step is to add table existence verification in `SessionCatalog` - [X] The fifth step is to add function existence verification in `SessionCatalog` Add test cases and verify the error messages we issued Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #12385 from gatorsmile/verifySessionAPIs.
* [SPARK-15037][SQL][MLLIB] Use SparkSession instead of SQLContext in ↵Sandeep Singh2016-05-10224-2611/+2934
| | | | | | | | | | | | | | | Scala/Java TestSuites ## What changes were proposed in this pull request? Use SparkSession instead of SQLContext in Scala/Java TestSuites as this PR already very big working Python TestSuites in a diff PR. ## How was this patch tested? Existing tests Author: Sandeep Singh <sandeep@techaddict.me> Closes #12907 from techaddict/SPARK-15037.
* [SPARK-12837][CORE] reduce network IO for accumulatorsWenchen Fan2016-05-107-21/+51
| | | | | | | | | | Sending un-updated accumulators back to driver makes no sense, as merging a zero value accumulator is a no-op. We should only send back updated accumulators, to save network IO. new test in `TaskContextSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #12899 from cloud-fan/acc.
* [SPARK-11249][LAUNCHER] Throw error if app resource is not provided.Marcelo Vanzin2016-05-107-24/+38
| | | | | | | | | | | | | | | Without this, the code would build an invalid spark-submit command line, and a more cryptic error would be presented to the user. Also, expose a constant that allows users to set a dummy resource in cases where they don't need an actual resource file; for backwards compatibility, that uses the same "spark-internal" resource that Spark itself uses. Tested via unit tests, run-example, spark-shell, and running the thrift server with mixed spark and hive command line arguments. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #12909 from vanzin/SPARK-11249.
* [SPARK-13670][LAUNCHER] Propagate error from launcher to shell.Marcelo Vanzin2016-05-101-1/+18
| | | | | | | | | | | | | | bash doesn't really propagate errors from subshells when using redirection the way spark-class does; so, instead, this change captures the exit code of the launcher process in the command array, and checks it before executing the actual command. Tested by injecting an error in Main.java (the launcher entry point) and verifying the shell gets the right exit code from spark-class. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #12910 from vanzin/SPARK-13670.
* [SPARK-13382][DOCS][PYSPARK] Update pyspark testing notes in build docsHolden Karau2016-05-101-0/+15
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current build documents don't specify that for PySpark tests we need to include Hive in the assembly otherwise the ORC tests fail. ## How was the this patch tested? Manually built the docs locally. Ran the provided build command follow by the PySpark SQL tests. ![pyspark2](https://cloud.githubusercontent.com/assets/59893/13190008/8829cde4-d70f-11e5-8ff5-a88b7894d2ad.png) Author: Holden Karau <holden@us.ibm.com> Closes #11278 from holdenk/SPARK-13382-update-pyspark-testing-notes-r2.
* [SPARK-14773] [SPARK-15179] [SQL] Fix SQL building and enable Hive testsHerman van Hovell2016-05-1062-8/+2237
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes SQL building for predicate subqueries and correlated scalar subqueries. It also enables most Hive subquery tests. ## How was this patch tested? Enabled new tests in HiveComparisionSuite. Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #12988 from hvanhovell/SPARK-14773.
* [SPARK-15154] [SQL] Change key types to Long in testsPete Robbins2016-05-101-14/+14
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? As reported in the Jira the 2 tests changed here are using a key of type Integer where the Spark sql code assumes the type is Long. This PR changes the tests to use the correct key types. ## How was this patch tested? Test builds run on both Big Endian and Little Endian platforms Author: Pete Robbins <robbinspg@gmail.com> Closes #13009 from robbinspg/HashedRelationSuiteFix.
* [SPARK-14127][SQL] "DESC <table>": Extracts schema information from table ↵Cheng Lian2016-05-103-20/+183
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | properties for data source tables ## What changes were proposed in this pull request? This is a follow-up of #12934 and #12844. This PR adds a set of utility methods in `DDLUtils` to help extract schema information (user-defined schema, partition columns, and bucketing information) from data source table properties. These utility methods are then used in `DescribeTableCommand` to refine output for data source tables. Before this PR, the aforementioned schema information are only shown as table properties, which are hard to read. Sample output: ``` +----------------------------+---------------------------------------------------------+-------+ |col_name |data_type |comment| +----------------------------+---------------------------------------------------------+-------+ |a |bigint | | |b |bigint | | |c |bigint | | |d |bigint | | |# Partition Information | | | |# col_name | | | |d | | | | | | | |# Detailed Table Information| | | |Database: |default | | |Owner: |lian | | |Create Time: |Tue May 10 03:20:34 PDT 2016 | | |Last Access Time: |Wed Dec 31 16:00:00 PST 1969 | | |Location: |file:/Users/lian/local/src/spark/workspace-a/target/... | | |Table Type: |MANAGED | | |Table Parameters: | | | | rawDataSize |-1 | | | numFiles |1 | | | transient_lastDdlTime |1462875634 | | | totalSize |684 | | | spark.sql.sources.provider|parquet | | | EXTERNAL |FALSE | | | COLUMN_STATS_ACCURATE |false | | | numRows |-1 | | | | | | |# Storage Information | | | |SerDe Library: |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe | | |InputFormat: |org.apache.hadoop.mapred.SequenceFileInputFormat | | |OutputFormat: |org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat| | |Compressed: |No | | |Num Buckets: |2 | | |Bucket Columns: |[b] | | |Sort Columns: |[c] | | |Storage Desc Parameters: | | | | path |file:/Users/lian/local/src/spark/workspace-a/target/... | | | serialization.format |1 | | +----------------------------+---------------------------------------------------------+-------+ ``` ## How was this patch tested? Test cases are added in `HiveDDLSuite` to check command output. Author: Cheng Lian <lian@databricks.com> Closes #13025 from liancheng/spark-14127-extract-schema-info.
* [SPARK-14963][YARN] Using recoveryPath if NM recovery is enabledjerryshao2016-05-102-22/+143
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? From Hadoop 2.5+, Yarn NM supports NM recovery which using recovery path for auxiliary services such as spark_shuffle, mapreduce_shuffle. So here change to use this path install of NM local dir if NM recovery is enabled. ## How was this patch tested? Unit test + local test. Author: jerryshao <sshao@hortonworks.com> Closes #12994 from jerryshao/SPARK-14963.
* [SPARK-14542][CORE] PipeRDD should allow configurable buffer size for…Sital Kedia2016-05-105-27/+54
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently PipedRDD internally uses PrintWriter to write data to the stdin of the piped process, which by default uses a BufferedWriter of buffer size 8k. In our experiment, we have seen that 8k buffer size is too small and the job spends significant amount of CPU time in system calls to copy the data. We should have a way to configure the buffer size for the writer. ## How was this patch tested? Ran PipedRDDSuite tests. Author: Sital Kedia <skedia@fb.com> Closes #12309 from sitalkedia/bufferedPipedRDD.
* [SPARK-15215][SQL] Fix Explain Parsing and Outputgatorsmile2016-05-106-29/+24
| | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? This PR is to address a few existing issues in `EXPLAIN`: - The `EXPLAIN` options `LOGICAL | FORMATTED | EXTENDED | CODEGEN` should not be 0 or more match. It should 0 or one match. Parser does not allow users to use more than one option in a single command. - The option `LOGICAL` is not supported. Issue an exception when users specify this option in the command. - The output of `EXPLAIN ` contains a weird empty line when the output of analyzed plan is empty. We should remove it. For example: ``` == Parsed Logical Plan == CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io. HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false == Analyzed Logical Plan == CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io. HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false == Optimized Logical Plan == CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io. HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false ... ``` #### How was this patch tested? Added and modified a few test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #12991 from gatorsmile/explainCreateTable.
* [SPARK-15187][SQL] Disallow Dropping Default Databasegatorsmile2016-05-104-52/+106
| | | | | | | | | | | | | | #### What changes were proposed in this pull request? In Hive Metastore, dropping default database is not allowed. However, in `InMemoryCatalog`, this is allowed. This PR is to disallow users to drop default database. #### How was this patch tested? Previously, we already have a test case in HiveDDLSuite. Now, we also add the same one in DDLSuite Author: gatorsmile <gatorsmile@gmail.com> Closes #12962 from gatorsmile/dropDefaultDB.
* [SPARK-15229][SQL] Make case sensitivity setting internalReynold Xin2016-05-091-1/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? Our case sensitivity support is different from what ANSI SQL standards support. Postgres' behavior is that if an identifier is quoted, then it is treated as case sensitive; otherwise it is folded to lowercase. We will likely need to revisit this in the future and change our behavior. For now, the safest change to do for Spark 2.0 is to make the case sensitive option internal and discourage users from turning it on, effectively making Spark always case insensitive. ## How was this patch tested? N/A - a small config documentation change. Author: Reynold Xin <rxin@databricks.com> Closes #13011 from rxin/SPARK-15229.
* [SPARK-15234][SQL] Fix spark.catalog.listDatabases.show()Andrew Or2016-05-094-14/+55
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Before: ``` scala> spark.catalog.listDatabases.show() +--------------------+-----------+-----------+ | name|description|locationUri| +--------------------+-----------+-----------+ |Database[name='de...| |Database[name='my...| |Database[name='so...| +--------------------+-----------+-----------+ ``` After: ``` +-------+--------------------+--------------------+ | name| description| locationUri| +-------+--------------------+--------------------+ |default|Default Hive data...|file:/user/hive/w...| | my_db| This is a database|file:/Users/andre...| |some_db| |file:/private/var...| +-------+--------------------+--------------------+ ``` ## How was this patch tested? New test in `CatalogSuite` Author: Andrew Or <andrew@databricks.com> Closes #13015 from andrewor14/catalog-show.
* [SPARK-15025][SQL] fix duplicate of PATH key in datasource table optionsxin Wu2016-05-092-6/+29
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The issue is that when the user provides the path option with uppercase "PATH" key, `options` contains `PATH` key and will get into the non-external case in the following code in `createDataSourceTables.scala`, where a new key "path" is created with a default path. ``` val optionsWithPath = if (!options.contains("path")) { isExternal = false options + ("path" -> sessionState.catalog.defaultTablePath(tableIdent)) } else { options } ``` So before creating hive table, serdeInfo.parameters will contain both "PATH" and "path" keys and different directories. and Hive table's dataLocation contains the value of "path". The fix in this PR is to convert `options` in the code above to `CaseInsensitiveMap` before checking for containing "path" key. ## How was this patch tested? A testcase is added Author: xin Wu <xinwu@us.ibm.com> Closes #12804 from xwu0226/SPARK-15025.
* [SPARK-15209] Fix display of job descriptions with single quotes in web UI ↵Josh Rosen2016-05-102-8/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | timeline ## What changes were proposed in this pull request? This patch fixes an escaping bug in the Web UI's event timeline that caused Javascript errors when displaying timeline entries whose descriptions include single quotes. The original bug can be reproduced by running ```scala sc.setJobDescription("double quote: \" ") sc.parallelize(1 to 10).count() sc.setJobDescription("single quote: ' ") sc.parallelize(1 to 10).count() ``` and then browsing to the driver UI. Previously, this resulted in an "Uncaught SyntaxError" because the single quote from the description was not escaped and ended up closing a Javascript string literal too early. The fix implemented here is to change the relevant Javascript to define its string literals using double-quotes. Our escaping logic already properly escapes double quotes in the description, so this is safe to do. ## How was this patch tested? Tested manually in `spark-shell` using the following cases: ```scala sc.setJobDescription("double quote: \" ") sc.parallelize(1 to 10).count() sc.setJobDescription("single quote: ' ") sc.parallelize(1 to 10).count() sc.setJobDescription("ampersand: &") sc.parallelize(1 to 10).count() sc.setJobDescription("newline: \n text after newline ") sc.parallelize(1 to 10).count() sc.setJobDescription("carriage return: \r text after return ") sc.parallelize(1 to 10).count() ``` /cc sarutak for review. Author: Josh Rosen <joshrosen@databricks.com> Closes #12995 from JoshRosen/SPARK-15209.
* [SPARK-14972] Improve performance of JSON schema inference's compatibleType ↵Josh Rosen2016-05-094-24/+94
| | | | | | | | | | | | | | | | | | method This patch improves the performance of `InferSchema.compatibleType` and `inferField`. The net result of this patch is a 6x speedup in local benchmarks running against cached data with a massive nested schema. The key idea is to remove unnecessary sorting in `compatibleType`'s `StructType` merging code. This code takes two structs, merges the fields with matching names, and copies over the unique fields, producing a new schema which is the union of the two structs' schemas. Previously, this code performed a very inefficient `groupBy()` to match up fields with the same name, but this is unnecessary because `inferField` already sorts structs' fields by name: since both lists of fields are sorted, we can simply merge them in a single pass. This patch also speeds up the existing field sorting in `inferField`: the old sorting code allocated unnecessary intermediate collections, while the new code uses mutable collects and performs in-place sorting. I rewrote inefficient `equals()` implementations in `StructType` and `Metadata`, significantly reducing object allocations in those methods. Finally, I replaced a `treeAggregate` call with `fold`: I doubt that `treeAggregate` will benefit us very much because the schemas would have to be enormous to realize large savings in network traffic. Since most schemas are probably fairly small in serialized form, they should typically fit within a direct task result and therefore can be incrementally merged at the driver as individual tasks finish. This change eliminates an entire (short) scheduler stage. Author: Josh Rosen <joshrosen@databricks.com> Closes #12750 from JoshRosen/schema-inference-speedups.
* [SPARK-15173][SQL] DataFrameWriter.insertInto should work with datasource ↵Wenchen Fan2016-05-094-7/+23
| | | | | | | | | | | | table stored in hive When we parse `CREATE TABLE USING`, we should build a `CreateTableUsing` plan with the `managedIfNoPath` set to true. Then we will add default table path to options when write it to hive. new test in `SQLQuerySuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #12949 from cloud-fan/bug.