aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [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.
* [SPARK-14459][SQL] Detect relation partitioning and adjust the logical planRyan Blue2016-05-095-12/+143
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This detects a relation's partitioning and adds checks to the analyzer. If an InsertIntoTable node has no partitioning, it is replaced by the relation's partition scheme and input columns are correctly adjusted, placing the partition columns at the end in partition order. If an InsertIntoTable node has partitioning, it is checked against the table's reported partitions. These changes required adding a PartitionedRelation trait to the catalog interface because Hive's MetastoreRelation doesn't extend CatalogRelation. This commit also includes a fix to InsertIntoTable's resolved logic, which now detects that all expected columns are present, including dynamic partition columns. Previously, the number of expected columns was not checked and resolved was true if there were missing columns. ## How was this patch tested? This adds new tests to the InsertIntoTableSuite that are fixed by this PR. Author: Ryan Blue <blue@apache.org> Closes #12239 from rdblue/SPARK-14459-detect-hive-partitioning.
* [SPARK-15184][SQL] Fix Silent Removal of An Existent Temp Table by Rename Tablegatorsmile2016-05-092-0/+69
| | | | | | | | | | | | | | #### What changes were proposed in this pull request? Currently, if we rename a temp table `Tab1` to another existent temp table `Tab2`. `Tab2` will be silently removed. This PR is to detect it and issue an exception message. In addition, this PR also detects another issue in the rename table command. When the destination table identifier does have database name, we should not ignore them. That might mean users could rename a regular table. #### How was this patch tested? Added two related test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #12959 from gatorsmile/rewriteTable.
* [SPARK-15185][SQL] InMemoryCatalog: Silent Removal of an Existent ↵gatorsmile2016-05-092-5/+62
| | | | | | | | | | | | | | | | Table/Function/Partitions by Rename #### What changes were proposed in this pull request? So far, in the implementation of InMemoryCatalog, we do not check if the new/destination table/function/partition exists or not. Thus, we just silently remove the existent table/function/partition. This PR is to detect them and issue an appropriate exception. #### How was this patch tested? Added the related test cases. They also verify if HiveExternalCatalog also detects these errors. Author: gatorsmile <gatorsmile@gmail.com> Closes #12960 from gatorsmile/renameInMemoryCatalog.
* [SPARK-15122] [SQL] Fix TPC-DS 41 - Normalize predicates before pulling them outHerman van Hovell2016-05-062-1/+15
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The official TPC-DS 41 query currently fails because it contains a scalar subquery with a disjunctive correlated predicate (the correlated predicates were nested in ORs). This makes the `Analyzer` pull out the entire predicate which is wrong and causes the following (correct) analysis exception: `The correlated scalar subquery can only contain equality predicates` This PR fixes this by first simplifing (or normalizing) the correlated predicates before pulling them out of the subquery. ## How was this patch tested? Manual testing on TPC-DS 41, and added a test to SubquerySuite. Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #12954 from hvanhovell/SPARK-15122.
* [SPARK-15051][SQL] Create a TypedColumn aliasKevin Yu2016-05-072-6/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently when we create an alias against a TypedColumn from user-defined Aggregator(for example: agg(aggSum.toColumn as "a")), spark is using the alias' function from Column( as), the alias function will return a column contains a TypedAggregateExpression, which is unresolved because the inputDeserializer is not defined. Later the aggregator function (agg) will inject the inputDeserializer back to the TypedAggregateExpression, but only if the aggregate columns are TypedColumn, in the above case, the TypedAggregateExpression will remain unresolved because it is under column and caused the problem reported by this jira [15051](https://issues.apache.org/jira/browse/SPARK-15051?jql=project%20%3D%20SPARK). This PR propose to create an alias function for TypedColumn, it will return a TypedColumn. It is using the similar code path as Column's alia function. For the spark build in aggregate function, like max, it is working with alias, for example val df1 = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") checkAnswer(df1.agg(max("j") as "b"), Row(3) :: Nil) Thanks for comments. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Add test cases in DatasetAggregatorSuite.scala run the sql related queries against this patch. Author: Kevin Yu <qyu@us.ibm.com> Closes #12893 from kevinyu98/spark-15051.
* [SPARK-14997][SQL] Fixed FileCatalog to return correct set of files when ↵Tathagata Das2016-05-065-30/+356
| | | | | | | | | | | | | | | | | | | | | | | | | | there is no partitioning scheme in the given paths ## What changes were proposed in this pull request? Lets says there are json files in the following directories structure ``` xyz/file0.json xyz/subdir1/file1.json xyz/subdir2/file2.json xyz/subdir1/subsubdir1/file3.json ``` `sqlContext.read.json("xyz")` should read only file0.json according to behavior in Spark 1.6.1. However in current master, all the 4 files are read. The fix is to make FileCatalog return only the children files of the given path if there is not partitioning detected (instead of all the recursive list of files). Closes #12774 ## How was this patch tested? unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #12856 from tdas/SPARK-14997.
* [SPARK-15108][SQL] Describe Permanent UDTFgatorsmile2016-05-0611-31/+91
| | | | | | | | | | | | | | #### What changes were proposed in this pull request? When Describe a UDTF, the command returns a wrong result. The command is unable to find the function, which has been created and cataloged in the catalog but not in the functionRegistry. This PR is to correct it. If the function is not in the functionRegistry, we will check the catalog for collecting the information of the UDTF function. #### How was this patch tested? Added test cases to verify the results Author: gatorsmile <gatorsmile@gmail.com> Closes #12885 from gatorsmile/showFunction.
* [SPARK-14962][SQL] Do not push down isnotnull/isnull on unsuportted types in ORChyukjinkwon2016-05-076-56/+126
| | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14962 ORC filters were being pushed down for all types for both `IsNull` and `IsNotNull`. This is apparently OK because both `IsNull` and `IsNotNull` do not take a type as an argument (Hive 1.2.x) during building filters (`SearchArgument`) in Spark-side but they do not filter correctly because stored statistics always produces `null` for not supported types (eg `ArrayType`) in ORC-side. So, it is always `true` for `IsNull` which ends up with always `false` for `IsNotNull`. (Please see [RecordReaderImpl.java#L296-L318](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L296-L318) and [RecordReaderImpl.java#L359-L365](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L359-L365) in Hive 1.2) This looks prevented in Hive 1.3.x >= by forcing to give a type ([`PredicateLeaf.Type`](https://github.com/apache/hive/blob/e085b7e9bd059d91aaf013df0db4d71dca90ec6f/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java#L50-L56)) when building a filter ([`SearchArgument`](https://github.com/apache/hive/blob/26b5c7b56a4f28ce3eabc0207566cce46b29b558/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java#L260)) but Hive 1.2.x seems not doing this. This PR prevents ORC filter creation for `IsNull` and `IsNotNull` on unsupported types. `OrcFilters` resembles `ParquetFilters`. ## How was this patch tested? Unittests in `OrcQuerySuite` and `OrcFilterSuite` and `sbt scalastyle`. Author: hyukjinkwon <gurwls223@gmail.com> Author: Hyukjin Kwon <gurwls223@gmail.com> Closes #12777 from HyukjinKwon/SPARK-14962.
* [SPARK-15152][DOC][MINOR] Scaladoc and Code style ImprovementsJacek Laskowski2016-05-057-15/+20
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Minor doc and code style fixes ## How was this patch tested? local build Author: Jacek Laskowski <jacek@japila.pl> Closes #12928 from jaceklaskowski/SPARK-15152.
* [SPARK-14893][SQL] Re-enable HiveSparkSubmitSuite SPARK-8489 test after ↵Dilip Biswal2016-05-054-7/+8
| | | | | | | | | | | | | | | | HiveContext is removed ## What changes were proposed in this pull request? Enable the test that was disabled when HiveContext was removed. ## How was this patch tested? Made sure the enabled test passes with the new jar. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #12924 from dilipbiswal/spark-14893.
* [SPARK-15134][EXAMPLE] Indent SparkSession builder patterns and update ↵Dongjoon Hyun2016-05-054-14/+15
| | | | | | | | | | | | | | | | | | | binary_classification_metrics_example.py ## What changes were proposed in this pull request? This issue addresses the comments in SPARK-15031 and also fix java-linter errors. - Use multiline format in SparkSession builder patterns. - Update `binary_classification_metrics_example.py` to use `SparkSession`. - Fix Java Linter errors (in SPARK-13745, SPARK-15031, and so far) ## How was this patch tested? After passing the Jenkins tests and run `dev/lint-java` manually. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12911 from dongjoon-hyun/SPARK-15134.
* [SPARK-15135][SQL] Make sure SparkSession thread safeShixiong Zhu2016-05-056-56/+73
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Went through SparkSession and its members and fixed non-thread-safe classes used by SparkSession ## How was this patch tested? Existing unit tests Author: Shixiong Zhu <shixiong@databricks.com> Closes #12915 from zsxwing/spark-session-thread-safe.
* [SPARK-15072][SQL][REPL][EXAMPLES] Remove SparkSession.withHiveSupportSandeep Singh2016-05-053-19/+11
| | | | | | | | | | | | ## What changes were proposed in this pull request? Removing the `withHiveSupport` method of `SparkSession`, instead use `enableHiveSupport` ## How was this patch tested? ran tests locally Author: Sandeep Singh <sandeep@techaddict.me> Closes #12851 from techaddict/SPARK-15072.
* [SPARK-14124][SQL][FOLLOWUP] Implement Database-related DDL Commandsgatorsmile2016-05-054-97/+311
| | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? First, a few test cases failed in mac OS X because the property value of `java.io.tmpdir` does not include a trailing slash on some platform. Hive always removes the last trailing slash. For example, what I got in the web: ``` Win NT --> C:\TEMP\ Win XP --> C:\TEMP Solaris --> /var/tmp/ Linux --> /var/tmp ``` Second, a couple of test cases are added to verify if the commands work properly. #### How was this patch tested? Added a test case for it and correct the previous test cases. Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #12081 from gatorsmile/mkdir.
* [SPARK-15110] [SPARKR] Implement repartitionByColumn for SparkR DataFramesNarineK2016-05-051-2/+3
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Implement repartitionByColumn on DataFrame. This will allow us to run R functions on each partition identified by column groups with dapply() method. ## How was this patch tested? Unit tests Author: NarineK <narine.kokhlikyan@us.ibm.com> Closes #12887 from NarineK/repartitionByColumns.
* [SPARK-15148][SQL] Upgrade Univocity library from 2.0.2 to 2.1.0hyukjinkwon2016-05-052-3/+3
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-15148 Mainly it improves the performance roughtly about 30%-40% according to the [release note](https://github.com/uniVocity/univocity-parsers/releases/tag/v2.1.0). For the details of the purpose is described in the JIRA. This PR upgrades Univocity library from 2.0.2 to 2.1.0. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #12923 from HyukjinKwon/SPARK-15148.
* [SPARK-14139][SQL] RowEncoder should preserve schema nullabilityWenchen Fan2016-05-064-16/+88
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The problem is: In `RowEncoder`, we use `Invoke` to get the field of an external row, which lose the nullability information. This PR creates a `GetExternalRowField` expression, so that we can preserve the nullability info. TODO: simplify the null handling logic in `RowEncoder`, to remove so many if branches, in follow-up PR. ## How was this patch tested? new tests in `RowEncoderSuite` Note that, This PR takes over https://github.com/apache/spark/pull/11980, with a little simplification, so all credits should go to koertkuipers Author: Wenchen Fan <wenchen@databricks.com> Author: Koert Kuipers <koert@tresata.com> Closes #12364 from cloud-fan/nullable.