aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-14897][SQL] upgrade to jetty 9.2.16bomeng2016-05-121-9/+7
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Since Jetty 8 is EOL (end of life) and has critical security issue [http://www.securityweek.com/critical-vulnerability-found-jetty-web-server], I think upgrading to 9 is necessary. I am using latest 9.2 since 9.3 requires Java 8+. `javax.servlet` and `derby` were also upgraded since Jetty 9.2 needs corresponding version. ## How was this patch tested? Manual test and current test cases should cover it. Author: bomeng <bmeng@us.ibm.com> Closes #12916 from bomeng/SPARK-14897.
* [SPARK-14684][SPARK-15277][SQL] Partition Spec Validation in SessionCatalog ↵gatorsmile2016-05-126-58/+248
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | and Checking Partition Spec Existence Before Dropping #### What changes were proposed in this pull request? ~~Currently, multiple partitions are allowed to drop by using a single DDL command: Alter Table Drop Partition. However, the internal implementation could break atomicity. That means, we could just drop a subset of qualified partitions, if hitting an exception when dropping one of qualified partitions~~ ~~This PR contains the following behavior changes:~~ ~~- disallow dropping multiple partitions by a single command ~~ ~~- allow users to input predicates in partition specification and issue a nicer error message if the predicate's comparison operator is not `=`.~~ ~~- verify the partition spec in SessionCatalog. This can ensure each partition spec in `Drop Partition` does not correspond to multiple partitions.~~ This PR has two major parts: - Verify the partition spec in SessionCatalog for fixing the following issue: ```scala sql(s"ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-09', unknownCol='12')") ``` Above example uses an invalid partition spec. Without this PR, we will drop all the partitions. The reason is Hive megastores getPartitions API returns all the partitions if we provide an invalid spec. - Re-implemented the `dropPartitions` in `HiveClientImpl`. Now, we always check if all the user-specified partition specs exist before attempting to drop the partitions. Previously, we start drop the partition before completing checking the existence of all the partition specs. If any failure happened after we start to drop the partitions, we will log an error message to indicate which partitions have been dropped and which partitions have not been dropped. #### How was this patch tested? Modified the existing test cases and added new test cases. Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #12801 from gatorsmile/banDropMultiPart.
* [SPARK-15094][SPARK-14803][SQL] Remove extra Project added in ↵Liang-Chi Hsieh2016-05-122-10/+62
| | | | | | | | | | | | | | | | | EliminateSerialization ## What changes were proposed in this pull request? We will eliminate the pair of `DeserializeToObject` and `SerializeFromObject` in `Optimizer` and add extra `Project`. However, when DeserializeToObject's outputObjectType is ObjectType and its cls can't be processed by unsafe project, it will be failed. To fix it, we can simply remove the extra `Project` and replace the output attribute of `DeserializeToObject` in another rule. ## How was this patch tested? `DatasetSuite`. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #12926 from viirya/fix-eliminate-serialization-projection.
* [SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempViewSean Zhong2016-05-1233-78/+123
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Deprecates registerTempTable and add dataset.createTempView, dataset.createOrReplaceTempView. ## How was this patch tested? Unit tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #12945 from clockfly/spark-15171.
* [SPARK-15160][SQL] support data source table in InMemoryCatalogWenchen Fan2016-05-118-67/+114
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds a new rule to convert `SimpleCatalogRelation` to data source table if its table property contains data source information. ## How was this patch tested? new test in SQLQuerySuite Author: Wenchen Fan <wenchen@databricks.com> Closes #12935 from cloud-fan/ds-table.
* [SPARK-14346] SHOW CREATE TABLE for data source tablesCheng Lian2016-05-1114-127/+458
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds native `SHOW CREATE TABLE` DDL command for data source tables. Support for Hive tables will be added in follow-up PR(s). To show table creation DDL for data source tables created by CTAS statements, this PR also added partitioning and bucketing support for normal `CREATE TABLE ... USING ...` syntax. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) A new test suite `ShowCreateTableSuite` is added in sql/hive package to test the new feature. Author: Cheng Lian <lian@databricks.com> Closes #12781 from liancheng/spark-14346-show-create-table.
* [SPARK-15080][CORE] Break copyAndReset into copy and resetSandeep Singh2016-05-121-4/+9
| | | | | | | | | | | | ## What changes were proposed in this pull request? Break copyAndReset into two methods copy and reset instead of just one. ## How was this patch tested? Existing Tests Author: Sandeep Singh <sandeep@techaddict.me> Closes #12936 from techaddict/SPARK-15080.
* [SPARK-15072][SQL][PYSPARK] FollowUp: Remove SparkSession.withHiveSupport in ↵Sandeep Singh2016-05-111-1/+1
| | | | | | | | | | | | | | | PySpark ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/12851 Remove `SparkSession.withHiveSupport` in PySpark and instead use `SparkSession.builder. enableHiveSupport` ## How was this patch tested? Existing tests. Author: Sandeep Singh <sandeep@techaddict.me> Closes #13063 from techaddict/SPARK-15072-followup.
* [SPARK-15264][SPARK-15274][SQL] CSV Reader Error on Blank Column NamesBill Chambers2016-05-113-4/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? When a CSV begins with: - `,,` OR - `"","",` meaning that the first column names are either empty or blank strings and `header` is specified to be `true`, then the column name is replaced with `C` + the index number of that given column. For example, if you were to read in the CSV: ``` "","second column" "hello", "there" ``` Then column names would become `"C0", "second column"`. This behavior aligns with what currently happens when `header` is specified to be `false` in recent versions of Spark. ### Current Behavior in Spark <=1.6 In Spark <=1.6, a CSV with a blank column name becomes a blank string, `""`, meaning that this column cannot be accessed. However the CSV reads in without issue. ### Current Behavior in Spark 2.0 Spark throws a NullPointerError and will not read in the file. #### Reproduction in 2.0 https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/346304/2828750690305044/484361/latest.html ## How was this patch tested? A new test was added to `CSVSuite` to account for this issue. We then have asserts that test for being able to select both the empty column names as well as the regular column names. Author: Bill Chambers <bill@databricks.com> Author: Bill Chambers <wchambers@ischool.berkeley.edu> Closes #13041 from anabranch/master.
* [SPARK-15276][SQL] CREATE TABLE with LOCATION should imply EXTERNALAndrew Or2016-05-114-14/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Before: ```sql -- uses that location but issues a warning CREATE TABLE my_tab LOCATION /some/path -- deletes any existing data in the specified location DROP TABLE my_tab ``` After: ```sql -- uses that location but creates an EXTERNAL table instead CREATE TABLE my_tab LOCATION /some/path -- does not delete the data at /some/path DROP TABLE my_tab ``` This patch essentially makes the `EXTERNAL` field optional. This is related to #13032. ## How was this patch tested? New test in `DDLCommandSuite`. Author: Andrew Or <andrew@databricks.com> Closes #13060 from andrewor14/location-implies-external.
* [SPARK-15257][SQL] Require CREATE EXTERNAL TABLE to specify LOCATIONAndrew Or2016-05-117-14/+31
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Before: ```sql -- uses warehouse dir anyway CREATE EXTERNAL TABLE my_tab -- doesn't actually delete the data DROP TABLE my_tab ``` After: ```sql -- no location is provided, throws exception CREATE EXTERNAL TABLE my_tab -- creates an external table using that location CREATE EXTERNAL TABLE my_tab LOCATION '/path/to/something' -- doesn't delete the data, which is expected DROP TABLE my_tab ``` ## How was this patch tested? New test in `DDLCommandSuite` Author: Andrew Or <andrew@databricks.com> Closes #13032 from andrewor14/create-external-table-location.
* [SPARK-15248][SQL] Make MetastoreFileCatalog consider directories from ↵Tathagata Das2016-05-113-11/+67
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | partition specs of a partitioned metastore table Table partitions can be added with locations different from default warehouse location of a hive table. `CREATE TABLE parquetTable (a int) PARTITIONED BY (b int) STORED AS parquet ` `ALTER TABLE parquetTable ADD PARTITION (b=1) LOCATION '/partition'` Querying such a table throws error as the MetastoreFileCatalog does not list the added partition directory, it only lists the default base location. ``` [info] - SPARK-15248: explicitly added partitions should be readable *** FAILED *** (1 second, 8 milliseconds) [info] java.util.NoSuchElementException: key not found: file:/Users/tdas/Projects/Spark/spark2/target/tmp/spark-b39ad224-c5d1-4966-8981-fb45a2066d61/partition [info] at scala.collection.MapLike$class.default(MapLike.scala:228) [info] at scala.collection.AbstractMap.default(Map.scala:59) [info] at scala.collection.MapLike$class.apply(MapLike.scala:141) [info] at scala.collection.AbstractMap.apply(Map.scala:59) [info] at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:59) [info] at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:55) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) [info] at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) [info] at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) [info] at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) [info] at scala.collection.AbstractTraversable.map(Traversable.scala:104) [info] at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog.listFiles(PartitioningAwareFileCatalog.scala:55) [info] at org.apache.spark.sql.execution.datasources.FileSourceStrategy$.apply(FileSourceStrategy.scala:93) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59) [info] at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434) [info] at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:55) [info] at org.apache.spark.sql.execution.SparkStrategies$SpecialLimits$.apply(SparkStrategies.scala:55) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59) [info] at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434) [info] at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60) [info] at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:77) [info] at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:75) [info] at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:82) [info] at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:82) [info] at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:330) [info] at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:146) [info] at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:159) [info] at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:554) [info] at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:535) [info] at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:125) [info] at org.apache.spark.sql.hive.ParquetPartitioningTest.withTempDir(parquetSuites.scala:726) [info] at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7.apply$mcV$sp(parquetSuites.scala:535) [info] at org.apache.spark.sql.test.SQLTestUtils$class.withTable(SQLTestUtils.scala:166) [info] at org.apache.spark.sql.hive.ParquetPartitioningTest.withTable(parquetSuites.scala:726) [info] at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply$mcV$sp(parquetSuites.scala:534) [info] at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534) [info] at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534) ``` The solution in this PR to get the paths to list from the partition spec and not rely on the default table path alone. unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13022 from tdas/SPARK-15248.
* [SPARK-15259] Sort time metric should not include spill and record insertion ↵Eric Liang2016-05-112-7/+9
| | | | | | | | | | | | | | | | | | | | | time ## What changes were proposed in this pull request? After SPARK-14669 it seems the sort time metric includes both spill and record insertion time. This makes it not very useful since the metric becomes close to the total execution time of the node. We should track just the time spent for in-memory sort, as before. ## How was this patch tested? Verified metric in the UI, also unit test on UnsafeExternalRowSorter. cc davies Author: Eric Liang <ekl@databricks.com> Author: Eric Liang <ekhliang@gmail.com> Closes #13035 from ericl/fix-metrics.
* [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-111-4/+4
| | | | | | | | | | | | | | ## 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-111-0/+5
| | | | | | | | | | | | | ## 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-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-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-101-32/+1
| | | | | | | | | | | | | | | ## 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-102-8/+14
| | | | | | | | | | | | ## 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-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-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-107-9/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-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-109-123/+259
| | | | | | | | | | | | | | | | | | 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-10110-1574/+1651
| | | | | | | | | | | | | | | 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-103-10/+10
| | | | | | | | | | 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-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-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-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.
* [SPARK-15166][SQL] Move some hive-specific code from SparkSessionAndrew Or2016-05-093-19/+13
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This also simplifies the code being moved. ## How was this patch tested? Existing tests. Author: Andrew Or <andrew@databricks.com> Closes #12941 from andrewor14/move-code.
* [SPARK-15210][SQL] Add missing @DeveloperApi annotation in sql.typesZheng RuiFeng2016-05-093-1/+6
| | | | | | | | | | add DeveloperApi annotation for `AbstractDataType` `MapType` `UserDefinedType` local build Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #12982 from zhengruifeng/types_devapi.
* [MINOR][SQL] Enhance the exception message if checkpointLocation is not setjerryshao2016-05-091-3/+9
| | | | | | | | | | | | | | | | | | | | | | | | | Enhance the exception message when `checkpointLocation` is not set, previously the message is: ``` java.util.NoSuchElementException: None.get at scala.None$.get(Option.scala:347) at scala.None$.get(Option.scala:345) at org.apache.spark.sql.DataFrameWriter$$anonfun$8.apply(DataFrameWriter.scala:338) at org.apache.spark.sql.DataFrameWriter$$anonfun$8.apply(DataFrameWriter.scala:338) at scala.collection.MapLike$class.getOrElse(MapLike.scala:128) at scala.collection.AbstractMap.getOrElse(Map.scala:59) at org.apache.spark.sql.DataFrameWriter.startStream(DataFrameWriter.scala:337) at org.apache.spark.sql.DataFrameWriter.startStream(DataFrameWriter.scala:277) ... 48 elided ``` This is not so meaningful, so changing to make it more specific. Local verified. Author: jerryshao <sshao@hortonworks.com> Closes #12998 from jerryshao/improve-exception-message.
* [SPARK-15225][SQL] Replace SQLContext with SparkSession in Encoder documentationLiang-Chi Hsieh2016-05-091-4/+4
| | | | | | | | | | `Encoder`'s doc mentions `sqlContext.implicits._`. We should use `sparkSession.implicits._` instead now. Only doc update. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #13002 from viirya/encoder-doc.
* [SPARK-14127][SQL] Makes 'DESC [EXTENDED|FORMATTED] <table>' support data ↵Cheng Lian2016-05-092-30/+47
| | | | | | | | | | | | | | | | source tables ## What changes were proposed in this pull request? This is a follow-up of PR #12844. It makes the newly updated `DescribeTableCommand` to support data sources tables. ## How was this patch tested? A test case is added to check `DESC [EXTENDED | FORMATTED] <table>` output. Author: Cheng Lian <lian@databricks.com> Closes #12934 from liancheng/spark-14127-desc-table-follow-up.
* [SPARK-15199][SQL] Disallow Dropping Build-in Functionsgatorsmile2016-05-092-1/+26
| | | | | | | | | | | | #### What changes were proposed in this pull request? As Hive and the major RDBMS behave, the built-in functions are not allowed to drop. In the current implementation, users can drop the built-in functions. However, after dropping the built-in functions, users are unable to add them back. #### How was this patch tested? Added a test case. Author: gatorsmile <gatorsmile@gmail.com> Closes #12975 from gatorsmile/dropBuildInFunction.
* [SPARK-15093][SQL] create/delete/rename directory for InMemoryCatalog ↵Wenchen Fan2016-05-094-44/+232
| | | | | | | | | | | | | | | | | | | | | | | | | operations if needed ## What changes were proposed in this pull request? following operations have file system operation now: 1. CREATE DATABASE: create a dir 2. DROP DATABASE: delete the dir 3. CREATE TABLE: create a dir 4. DROP TABLE: delete the dir 5. RENAME TABLE: rename the dir 6. CREATE PARTITIONS: create a dir 7. RENAME PARTITIONS: rename the dir 8. DROP PARTITIONS: drop the dir ## How was this patch tested? new tests in `ExternalCatalogSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #12871 from cloud-fan/catalog.