aboutsummaryrefslogtreecommitdiff
path: root/sql/hive/src/test
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-16690][TEST] rename SQLTestUtils.withTempTable to withTempViewWenchen Fan2016-07-2315-34/+34
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? after https://github.com/apache/spark/pull/12945, we renamed the `registerTempTable` to `createTempView`, as we do create a view actually. This PR renames `SQLTestUtils.withTempTable` to reflect this change. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14318 from cloud-fan/minor4.
* [SPARK-16344][SQL] Decoding Parquet array of struct with a single field ↵Cheng Lian2016-07-201-2/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | named "element" ## What changes were proposed in this pull request? Due to backward-compatibility reasons, the following Parquet schema is ambiguous: ``` optional group f (LIST) { repeated group list { optional group element { optional int32 element; } } } ``` According to the parquet-format spec, when interpreted as a standard 3-level layout, this type is equivalent to the following SQL type: ``` ARRAY<STRUCT<element: INT>> ``` However, when interpreted as a legacy 2-level layout, it's equivalent to ``` ARRAY<STRUCT<element: STRUCT<element: INT>>> ``` Historically, to disambiguate these cases, we employed two methods: - `ParquetSchemaConverter.isElementType()` Used to disambiguate the above cases while converting Parquet types to Spark types. - `ParquetRowConverter.isElementType()` Used to disambiguate the above cases while instantiating row converters that convert Parquet records to Spark rows. Unfortunately, these two methods make different decision about the above problematic Parquet type, and caused SPARK-16344. `ParquetRowConverter.isElementType()` is necessary for Spark 1.4 and earlier versions because Parquet requested schemata are directly converted from Spark schemata in these versions. The converted Parquet schemata may be incompatible with actual schemata of the underlying physical files when the files are written by a system/library that uses a schema conversion scheme that is different from Spark when writing Parquet LIST and MAP fields. In Spark 1.5, Parquet requested schemata are always properly tailored from schemata of physical files to be read. Thus `ParquetRowConverter.isElementType()` is no longer necessary. This PR replaces this method with a simply yet accurate scheme: whenever an ambiguous Parquet type is hit, convert the type in question back to a Spark type using `ParquetSchemaConverter` and check whether it matches the corresponding Spark type. ## How was this patch tested? New test cases added in `ParquetHiveCompatibilitySuite` and `ParquetQuerySuite`. Author: Cheng Lian <lian@databricks.com> Closes #14014 from liancheng/spark-16344-for-master-and-2.0.
* [HOTFIX] Fix Scala 2.10 compilationReynold Xin2016-07-181-2/+2
|
* [SPARK-16590][SQL] Improve LogicalPlanToSQLSuite to check generated SQL directlyDongjoon Hyun2016-07-18103-153/+820
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR improves `LogicalPlanToSQLSuite` to check the generated SQL directly by **structure**. So far, `LogicalPlanToSQLSuite` relies on `checkHiveQl` to ensure the **successful SQL generation** and **answer equality**. However, it does not guarantee the generated SQL is the same or will not be changed unnoticeably. ## How was this patch tested? Pass the Jenkins. This is only a testsuite change. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14235 from dongjoon-hyun/SPARK-16590.
* [SPARK-16515][SQL] set default record reader and writer for script ↵Daoyuan Wang2016-07-182-0/+34
| | | | | | | | | | | | | | | | | transformation ## What changes were proposed in this pull request? In ScriptInputOutputSchema, we read default RecordReader and RecordWriter from conf. Since Spark 2.0 has deleted those config keys from hive conf, we have to set default reader/writer class name by ourselves. Otherwise we will get None for LazySimpleSerde, the data written would not be able to read by script. The test case added worked fine with previous version of Spark, but would fail now. ## How was this patch tested? added a test case in SQLQuerySuite. Closes #14169 Author: Daoyuan Wang <daoyuan.wang@intel.com> Author: Yin Huai <yhuai@databricks.com> Closes #14249 from yhuai/scriptTransformation.
* [SPARK-16482][SQL] Describe Table Command for Tables Requiring Runtime ↵gatorsmile2016-07-131-7/+9
| | | | | | | | | | | | | | | | | | Inferred Schema #### What changes were proposed in this pull request? If we create a table pointing to a parquet/json datasets without specifying the schema, describe table command does not show the schema at all. It only shows `# Schema of this table is inferred at runtime`. In 1.6, describe table does show the schema of such a table. ~~For data source tables, to infer the schema, we need to load the data source tables at runtime. Thus, this PR calls the function `lookupRelation`.~~ For data source tables, we infer the schema before table creation. Thus, this PR set the inferred schema as the table schema when table creation. #### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14148 from gatorsmile/describeSchema.
* [SPARK-16284][SQL] Implement reflect SQL functionpetermaxlee2016-07-131-23/+0
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch implements reflect SQL function, which can be used to invoke a Java method in SQL. Slightly different from Hive, this implementation requires the class name and the method name to be literals. This implementation also supports only a smaller number of data types, and requires the function to be static, as suggested by rxin in #13969. java_method is an alias for reflect, so this should also resolve SPARK-16277. ## How was this patch tested? Added expression unit tests and an end-to-end test. Author: petermaxlee <petermaxlee@gmail.com> Closes #14138 from petermaxlee/reflect-static.
* [SPARK-16119][SQL] Support PURGE option to drop table / partition.Marcelo Vanzin2016-07-124-6/+31
| | | | | | | | | | | | | | | | | | | | | | | | | This option is used by Hive to directly delete the files instead of moving them to the trash. This is needed in certain configurations where moving the files does not work. For non-Hive tables and partitions, Spark already behaves as if the PURGE option was set, so there's no need to do anything. Hive support for PURGE was added in 0.14 (for tables) and 1.2 (for partitions), so the code reflects that: trying to use the option with older versions of Hive will cause an exception to be thrown. The change is a little noisier than I would like, because of the code to propagate the new flag through all the interfaces and implementations; the main changes are in the parser and in HiveShim, aside from the tests (DDLCommandSuite, VersionsSuite). Tested by running sql and catalyst unit tests, plus VersionsSuite which has been updated to test the version-specific behavior. I also ran an internal test suite that uses PURGE and would not pass previously. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #13831 from vanzin/SPARK-16119.
* [SPARK-15752][SQL] Optimize metadata only query that has an aggregate whose ↵Lianhui Wang2016-07-121-0/+89
| | | | | | | | | | | | | | | | children are deterministic project or filter operators. ## What changes were proposed in this pull request? when query only use metadata (example: partition key), it can return results based on metadata without scanning files. Hive did it in HIVE-1003. ## How was this patch tested? add unit tests Author: Lianhui Wang <lianhuiwang09@gmail.com> Author: Wenchen Fan <wenchen@databricks.com> Author: Lianhui Wang <lianhuiwang@users.noreply.github.com> Closes #13494 from lianhuiwang/metadata-only.
* [SPARK-12639][SQL] Mark Filters Fully Handled By Sources with *Russell Spitzer2016-07-111-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In order to make it clear which filters are fully handled by the underlying datasource we will mark them with an *. This will give a clear visual queue to users that the filter is being treated differently by catalyst than filters which are just presented to the underlying DataSource. Examples from the FilteredScanSuite, in this example `c IN (...)` is handled by the source, `b < ...` is not ### Before ``` //SELECT a FROM oneToTenFiltered WHERE a + b > 9 AND b < 16 AND c IN ('bbbbbBBBBB', 'cccccCCCCC', 'dddddDDDDD', 'foo') == Physical Plan == Project [a#0] +- Filter (((a#0 + b#1) > 9) && (b#1 < 16)) +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]] ``` ### After ``` == Physical Plan == Project [a#0] +- Filter (((a#0 + b#1) > 9) && (b#1 < 16)) +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), *In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]] ``` ## How was the this patch tested? Manually tested with the Spark Cassandra Connector, a source which fully handles underlying filters. Now fully handled filters appear with an * next to their names. I can add an automated test as well if requested Post 1.6.1 Tested by modifying the FilteredScanSuite to run explains. Author: Russell Spitzer <Russell.Spitzer@gmail.com> Closes #11317 from RussellSpitzer/SPARK-12639-Star.
* [SPARK-16459][SQL] Prevent dropping current databaseDongjoon Hyun2016-07-112-0/+3
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR prevents dropping the current database to avoid errors like the followings. ```scala scala> sql("create database delete_db") scala> sql("use delete_db") scala> sql("drop database delete_db") scala> sql("create table t as select 1") org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database `delete_db` not found; ``` ## How was this patch tested? Pass the Jenkins tests including an updated testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14115 from dongjoon-hyun/SPARK-16459.
* [SPARK-16415][SQL] fix catalog string errorDaoyuan Wang2016-07-071-3/+11
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? In #13537 we truncate `simpleString` if it is a long `StructType`. But sometimes we need `catalogString` to reconstruct `TypeInfo`, for example in description of [SPARK-16415 ](https://issues.apache.org/jira/browse/SPARK-16415). So we need to keep the implementation of `catalogString` not affected by our truncate. ## How was this patch tested? added a test case. Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #14089 from adrian-wang/catalogstring.
* [SPARK-16368][SQL] Fix Strange Errors When Creating View With Unmatched ↵gatorsmile2016-07-071-0/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | | | Column Num #### What changes were proposed in this pull request? When creating a view, a common user error is the number of columns produced by the `SELECT` clause does not match the number of column names specified by `CREATE VIEW`. For example, given Table `t1` only has 3 columns ```SQL create view v1(col2, col4, col3, col5) as select * from t1 ``` Currently, Spark SQL reports the following error: ``` requirement failed java.lang.IllegalArgumentException: requirement failed at scala.Predef$.require(Predef.scala:212) at org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:90) ``` This error message is very confusing. This PR is to detect the error and issue a meaningful error message. #### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14047 from gatorsmile/viewMismatchedColumns.
* [SPARK-16229][SQL] Drop Empty Table After CREATE TABLE AS SELECT failsgatorsmile2016-07-061-0/+15
| | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? In `CREATE TABLE AS SELECT`, if the `SELECT` query failed, the table should not exist. For example, ```SQL CREATE TABLE tab STORED AS TEXTFILE SELECT 1 AS a, (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b ``` The above query failed as expected but an empty table `t` is created. This PR is to drop the created table when hitting any non-fatal exception. #### How was this patch tested? Added a test case to verify the behavior Author: gatorsmile <gatorsmile@gmail.com> Closes #13926 from gatorsmile/dropTableAfterException.
* [SPARK-16388][SQL] Remove spark.sql.nativeView and ↵Reynold Xin2016-07-061-118/+88
| | | | | | | | | | | | | | spark.sql.nativeView.canonical config ## What changes were proposed in this pull request? These two configs should always be true after Spark 2.0. This patch removes them from the config list. Note that ideally this should've gone into branch-2.0, but due to the timing of the release we should only merge this in master for Spark 2.1. ## How was this patch tested? Updated test cases. Author: Reynold Xin <rxin@databricks.com> Closes #14061 from rxin/SPARK-16388.
* [SPARK-16383][SQL] Remove `SessionState.executeSql`Dongjoon Hyun2016-07-051-3/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR removes `SessionState.executeSql` in favor of `SparkSession.sql`. We can remove this safely since the visibility `SessionState` is `private[sql]` and `executeSql` is only used in one **ignored** test, `test("Multiple Hive Instances")`. ## How was this patch tested? Pass the Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14055 from dongjoon-hyun/SPARK-16383.
* [SPARK-16311][SQL] Metadata refresh should work on temporary viewsReynold Xin2016-07-053-11/+73
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes the bug that the refresh command does not work on temporary views. This patch is based on https://github.com/apache/spark/pull/13989, but removes the public Dataset.refresh() API as well as improved test coverage. Note that I actually think the public refresh() API is very useful. We can in the future implement it by also invalidating the lazy vals in QueryExecution (or alternatively just create a new QueryExecution). ## How was this patch tested? Re-enabled a previously ignored test, and added a new test suite for Hive testing behavior of temporary views against MetastoreRelation. Author: Reynold Xin <rxin@databricks.com> Author: petermaxlee <petermaxlee@gmail.com> Closes #14009 from rxin/SPARK-16311.
* [SPARK-15198][SQL] Support for pushing down filters for boolean types in ORC ↵hyukjinkwon2016-07-052-4/+34
| | | | | | | | | | | | | | | | | | | | data source ## What changes were proposed in this pull request? It seems ORC supports all the types in ([`PredicateLeaf.Type`](https://github.com/apache/hive/blob/e085b7e9bd059d91aaf013df0db4d71dca90ec6f/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java#L50-L56)) which includes boolean types. So, this was tested first. This PR adds the support for pushing filters down for `BooleanType` in ORC data source. This PR also removes `OrcTableScan` class and the companion object, which is not used anymore. ## How was this patch tested? Unittest in `OrcFilterSuite` and `OrcQuerySuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #12972 from HyukjinKwon/SPARK-15198.
* [SPARK-15968][SQL] Nonempty partitioned metastore tables are not cachedMichael Allman2016-07-051-18/+43
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | (Please note this is a revision of PR #13686, which has been closed in favor of this PR.) This PR addresses [SPARK-15968](https://issues.apache.org/jira/browse/SPARK-15968). ## What changes were proposed in this pull request? The `getCached` method of [HiveMetastoreCatalog](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala) computes `pathsInMetastore` from the metastore relation's catalog table. This only returns the table base path, which is incomplete/inaccurate for a nonempty partitioned table. As a result, cached lookups on nonempty partitioned tables always miss. Rather than get `pathsInMetastore` from metastoreRelation.catalogTable.storage.locationUri.toSeq I modified the `getCached` method to take a `pathsInMetastore` argument. Calls to this method pass in the paths computed from calls to the Hive metastore. This is how `getCached` was implemented in Spark 1.5: https://github.com/apache/spark/blob/e0c3212a9b42e3e704b070da4ac25b68c584427f/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala#L444. I also added a call in `InsertIntoHiveTable.scala` to invalidate the table from the SQL session catalog. ## How was this patch tested? I've added a new unit test to `parquetSuites.scala`: SPARK-15968: nonempty partitioned metastore Parquet table lookup should use cached relation Note that the only difference between this new test and the one above it in the file is that the new test populates its partitioned table with a single value, while the existing test leaves the table empty. This reveals a subtle, unexpected hole in test coverage present before this patch. Note I also modified a different but related unit test in `parquetSuites.scala`: SPARK-15248: explicitly added partitions should be readable This unit test asserts that Spark SQL should return data from a table partition which has been placed there outside a metastore query immediately after it is added. I changed the test so that, instead of adding the data as a parquet file saved in the partition's location, the data is added through a SQL `INSERT` query. I made this change because I could find no way to efficiently support partitioned table caching without failing that test. In addition to my primary motivation, I can offer a few reasons I believe this is an acceptable weakening of that test. First, it still validates a fix for [SPARK-15248](https://issues.apache.org/jira/browse/SPARK-15248), the issue for which it was written. Second, the assertion made is stronger than that required for non-partitioned tables. If you write data to the storage location of a non-partitioned metastore table without using a proper SQL DML query, a subsequent call to show that data will not return it. I believe this is an intentional limitation put in place to make table caching feasible, but I'm only speculating. Building a large `HadoopFsRelation` requires `stat`-ing all of its data files. In our environment, where we have tables with 10's of thousands of partitions, the difference between using a cached relation versus a new one is a matter of seconds versus minutes. Caching partitioned table metadata vastly improves the usability of Spark SQL for these cases. Thanks. Author: Michael Allman <michael@videoamp.com> Closes #13818 from mallman/spark-15968.
* [SPARK-16358][SQL] Remove InsertIntoHiveTable From Logical Plangatorsmile2016-07-041-2/+0
| | | | | | | | | | | | #### What changes were proposed in this pull request? LogicalPlan `InsertIntoHiveTable` is useless. Thus, we can remove it from the code base. #### How was this patch tested? The existing test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14037 from gatorsmile/InsertIntoHiveTable.
* [SPARK-16267][TEST] Replace deprecated `CREATE TEMPORARY TABLE ... USING` ↵Dongjoon Hyun2016-06-292-7/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | from testsuites. ## What changes were proposed in this pull request? After SPARK-15674, `DDLStrategy` prints out the following deprecation messages in the testsuites. ``` 12:10:53.284 WARN org.apache.spark.sql.execution.SparkStrategies$DDLStrategy: CREATE TEMPORARY TABLE normal_orc_source USING... is deprecated, please use CREATE TEMPORARY VIEW viewName USING... instead ``` Total : 40 - JDBCWriteSuite: 14 - DDLSuite: 6 - TableScanSuite: 6 - ParquetSourceSuite: 5 - OrcSourceSuite: 2 - SQLQuerySuite: 2 - HiveCommandSuite: 2 - JsonSuite: 1 - PrunedScanSuite: 1 - FilteredScanSuite 1 This PR replaces `CREATE TEMPORARY TABLE` with `CREATE TEMPORARY VIEW` in order to remove the deprecation messages in the above testsuites except `DDLSuite`, `SQLQuerySuite`, `HiveCommandSuite`. The Jenkins results shows only remaining 10 messages. https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61422/consoleFull ## How was this patch tested? This is a testsuite-only change. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13956 from dongjoon-hyun/SPARK-16267.
* [SPARK-16228][SQL] HiveSessionCatalog should return `double`-param functions ↵Dongjoon Hyun2016-06-291-0/+7
| | | | | | | | | | | | | | | | | | | | | | | | | for decimal param lookups ## What changes were proposed in this pull request? This PR supports a fallback lookup by casting `DecimalType` into `DoubleType` for the external functions with `double`-type parameter. **Reported Error Scenarios** ```scala scala> sql("select percentile(value, 0.5) from values 1,2,3 T(value)") org.apache.spark.sql.AnalysisException: ... No matching method for class org.apache.hadoop.hive.ql.udf.UDAFPercentile with (int, decimal(38,18)). Possible choices: _FUNC_(bigint, array<double>) _FUNC_(bigint, double) ; line 1 pos 7 scala> sql("select percentile_approx(value, 0.5) from values 1.0,2.0,3.0 T(value)") org.apache.spark.sql.AnalysisException: ... Only a float/double or float/double array argument is accepted as parameter 2, but decimal(38,18) was passed instead.; line 1 pos 7 ``` ## How was this patch tested? Pass the Jenkins tests (including a new testcase). Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13930 from dongjoon-hyun/SPARK-16228.
* [SPARK-16157][SQL] Add New Methods for comments in StructField and StructTypegatorsmile2016-06-291-0/+22
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? Based on the previous discussion with cloud-fan hvanhovell in another related PR https://github.com/apache/spark/pull/13764#discussion_r67994276, it looks reasonable to add convenience methods for users to add `comment` when defining `StructField`. Currently, the column-related `comment` attribute is stored in `Metadata` of `StructField`. For example, users can add the `comment` attribute using the following way: ```Scala StructType( StructField( "cl1", IntegerType, nullable = false, new MetadataBuilder().putString("comment", "test").build()) :: Nil) ``` This PR is to add more user friendly methods for the `comment` attribute when defining a `StructField`. After the changes, users are provided three different ways to do it: ```Scala val struct = (new StructType) .add("a", "int", true, "test1") val struct = (new StructType) .add("c", StringType, true, "test3") val struct = (new StructType) .add(StructField("d", StringType).withComment("test4")) ``` #### How was this patch tested? Added test cases: - `DataTypeSuite` is for testing three types of API changes, - `DataFrameReaderWriterSuite` is for parquet, json and csv formats - using in-memory catalog - `OrcQuerySuite.scala` is for orc format using Hive-metastore Author: gatorsmile <gatorsmile@gmail.com> Closes #13860 from gatorsmile/newMethodForComment.
* [SPARK-16220][SQL] Revert Change to Bring Back SHOW FUNCTIONS FunctionalityBill Chambers2016-06-271-0/+5
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Fix tests regarding show functions functionality - Revert `catalog.ListFunctions` and `SHOW FUNCTIONS` to return to `Spark 1.X` functionality. Cherry picked changes from this PR: https://github.com/apache/spark/pull/13413/files ## How was this patch tested? Unit tests. Author: Bill Chambers <bill@databricks.com> Author: Bill Chambers <wchambers@ischool.berkeley.edu> Closes #13916 from anabranch/master.
* [SPARK-13709][SQL] Initialize deserializer with both table and partition ↵Cheng Lian2016-06-231-0/+81
| | | | | | | | | | | | | | | | | | properties when reading partitioned tables ## What changes were proposed in this pull request? When reading partitions of a partitioned Hive SerDe table, we only initializes the deserializer using partition properties. However, for SerDes like `AvroSerDe`, essential properties (e.g. Avro schema information) may be defined in table properties. We should merge both table properties and partition properties before initializing the deserializer. Note that an individual partition may have different properties than the one defined in the table properties (e.g. partitions within a table can have different SerDes). Thus, for any property key defined in both partition and table properties, the value set in partition properties wins. ## How was this patch tested? New test case added in `QueryPartitionSuite`. Author: Cheng Lian <lian@databricks.com> Closes #13865 from liancheng/spark-13709-partitioned-avro-table.
* [SPARK-16024][SQL][TEST] Verify Column Comment for Data Source Tablesgatorsmile2016-06-231-0/+14
| | | | | | | | | | | | | | #### What changes were proposed in this pull request? This PR is to improve test coverage. It verifies whether `Comment` of `Column` can be appropriate handled. The test cases verify the related parts in Parser, both SQL and DataFrameWriter interface, and both Hive Metastore catalog and In-memory catalog. #### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #13764 from gatorsmile/dataSourceComment.
* [SPARK-15956][SQL] When unwrapping ORC avoid pattern matching at runtimeBrian Cho2016-06-221-0/+6
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Extend the returning of unwrapper functions from primitive types to all types. This PR is based on https://github.com/apache/spark/pull/13676. It only fixes a bug with scala-2.10 compilation. All credit should go to dafrista. ## How was this patch tested? The patch should pass all unit tests. Reading ORC files with non-primitive types with this change reduced the read time by ~15%. Author: Brian Cho <bcho@fb.com> Author: Herman van Hovell <hvanhovell@databricks.com> Closes #13854 from hvanhovell/SPARK-15956-scala210.
* [SPARK-15956][SQL] Revert "[] When unwrapping ORC avoid pattern matching…Herman van Hovell2016-06-221-6/+0
| | | | | | | | This reverts commit 0a9c02759515c41de37db6381750bc3a316c860c. It breaks the 2.10 build, I'll fix this in a different PR. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #13853 from hvanhovell/SPARK-15956-revert.
* [SPARK-15956][SQL] When unwrapping ORC avoid pattern matching at runtimeBrian Cho2016-06-221-0/+6
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Extend the returning of unwrapper functions from primitive types to all types. ## How was this patch tested? The patch should pass all unit tests. Reading ORC files with non-primitive types with this change reduced the read time by ~15%. === The github diff is very noisy. Attaching the screenshots below for improved readability: ![screen shot 2016-06-14 at 5 33 16 pm](https://cloud.githubusercontent.com/assets/1514239/16064580/4d6f7a98-3257-11e6-9172-65e4baff948b.png) ![screen shot 2016-06-14 at 5 33 28 pm](https://cloud.githubusercontent.com/assets/1514239/16064587/5ae6c244-3257-11e6-8460-69eee70de219.png) Author: Brian Cho <bcho@fb.com> Closes #13676 from dafrista/improve-orc-master.
* [SPARK-16037][SQL] Follow-up: add DataFrameWriter.insertInto() test cases ↵Cheng Lian2016-06-211-0/+48
| | | | | | | | | | | | | | | | | | for by position resolution ## What changes were proposed in this pull request? This PR migrates some test cases introduced in #12313 as a follow-up of #13754 and #13766. These test cases cover `DataFrameWriter.insertInto()`, while the former two only cover SQL `INSERT` statements. Note that the `testPartitionedTable` utility method tests both Hive SerDe tables and data source tables. ## How was this patch tested? N/A Author: Cheng Lian <lian@databricks.com> Closes #13810 from liancheng/spark-16037-follow-up-tests.
* [SPARK-16030][SQL] Allow specifying static partitions when inserting to data ↵Yin Huai2016-06-202-11/+88
| | | | | | | | | | | | | | | | source tables ## What changes were proposed in this pull request? This PR adds the static partition support to INSERT statement when the target table is a data source table. ## How was this patch tested? New tests in InsertIntoHiveTableSuite and DataSourceAnalysisSuite. **Note: This PR is based on https://github.com/apache/spark/pull/13766. The last commit is the actual change.** Author: Yin Huai <yhuai@databricks.com> Closes #13769 from yhuai/SPARK-16030-1.
* [SPARK-16036][SPARK-16037][SPARK-16034][SQL] Follow up code clean up and ↵Yin Huai2016-06-193-32/+55
| | | | | | | | | | | | | | improvement ## What changes were proposed in this pull request? This PR is the follow-up PR for https://github.com/apache/spark/pull/13754/files and https://github.com/apache/spark/pull/13749. I will comment inline to explain my changes. ## How was this patch tested? Existing tests. Author: Yin Huai <yhuai@databricks.com> Closes #13766 from yhuai/caseSensitivity.
* [SPARK-16036][SPARK-16037][SQL] fix various table insertion problemsWenchen Fan2016-06-184-56/+33
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current table insertion has some weird behaviours: 1. inserting into a partitioned table with mismatch columns has confusing error message for hive table, and wrong result for datasource table 2. inserting into a partitioned table without partition list has wrong result for hive table. This PR fixes these 2 problems. ## How was this patch tested? new test in hive `SQLQuerySuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #13754 from cloud-fan/insert2.
* [SPARK-16033][SQL] insertInto() can't be used together with partitionBy()Cheng Lian2016-06-171-0/+37
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? When inserting into an existing partitioned table, partitioning columns should always be determined by catalog metadata of the existing table to be inserted. Extra `partitionBy()` calls don't make sense, and mess up existing data because newly inserted data may have wrong partitioning directory layout. ## How was this patch tested? New test case added in `InsertIntoHiveTableSuite`. Author: Cheng Lian <lian@databricks.com> Closes #13747 from liancheng/spark-16033-insert-into-without-partition-by.
* [SPARK-15706][SQL] Fix Wrong Answer when using IF NOT EXISTS in INSERT ↵gatorsmile2016-06-161-0/+68
| | | | | | | | | | | | | | | | OVERWRITE for DYNAMIC PARTITION #### What changes were proposed in this pull request? `IF NOT EXISTS` in `INSERT OVERWRITE` should not support dynamic partitions. If we specify `IF NOT EXISTS`, the inserted statement is not shown in the table. This PR is to issue an exception in this case, just like what Hive does. Also issue an exception if users specify `IF NOT EXISTS` if users do not specify any `PARTITION` specification. #### How was this patch tested? Added test cases into `PlanParserSuite` and `InsertIntoHiveTableSuite` Author: gatorsmile <gatorsmile@gmail.com> Closes #13447 from gatorsmile/insertIfNotExist.
* [SPARK-15998][SQL] Verification of SQLConf HIVE_METASTORE_PARTITION_PRUNINGgatorsmile2016-06-161-3/+57
| | | | | | | | | | | | | | #### What changes were proposed in this pull request? `HIVE_METASTORE_PARTITION_PRUNING` is a public `SQLConf`. When `true`, some predicates will be pushed down into the Hive metastore so that unmatching partitions can be eliminated earlier. The current default value is `false`. For performance improvement, users might turn this parameter on. So far, the code base does not have such a test case to verify whether this `SQLConf` properly works. This PR is to improve the test case coverage for avoiding future regression. #### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #13716 from gatorsmile/addTestMetastorePartitionPruning.
* [SPARK-15862][SQL] Better Error Message When Having Database Name in CACHE ↵gatorsmile2016-06-161-19/+60
| | | | | | | | | | | | | | | | | | | | | | | | TABLE AS SELECT #### What changes were proposed in this pull request? ~~If the temp table already exists, we should not silently replace it when doing `CACHE TABLE AS SELECT`. This is inconsistent with the behavior of `CREAT VIEW` or `CREATE TABLE`. This PR is to fix this silent drop.~~ ~~Maybe, we also can introduce new syntax for replacing the existing one. For example, in Hive, to replace a view, the syntax should be like `ALTER VIEW AS SELECT` or `CREATE OR REPLACE VIEW AS SELECT`~~ The table name in `CACHE TABLE AS SELECT` should NOT contain database prefix like "database.table". Thus, this PR captures this in Parser and outputs a better error message, instead of reporting the view already exists. In addition, refactoring the `Parser` to generate table identifiers instead of returning the table name string. #### How was this patch tested? - Added a test case for caching and uncaching qualified table names - Fixed a few test cases that do not drop temp table at the end - Added the related test case for the issue resolved in this PR Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #13572 from gatorsmile/cacheTableAsSelect.
* [SPARK-15901][SQL][TEST] Verification of CONVERT_METASTORE_ORC and ↵gatorsmile2016-06-152-32/+83
| | | | | | | | | | | | | | CONVERT_METASTORE_PARQUET #### What changes were proposed in this pull request? So far, we do not have test cases for verifying whether the external parameters `HiveUtils .CONVERT_METASTORE_ORC` and `HiveUtils.CONVERT_METASTORE_PARQUET` properly works when users use non-default values. This PR is to add such test cases for avoiding potential regression. #### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #13622 from gatorsmile/addTestCase4parquetOrcConversion.
* [SPARK-15959][SQL] Add the support of hive.metastore.warehouse.dir backYin Huai2016-06-151-14/+77
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds the support of conf `hive.metastore.warehouse.dir` back. With this patch, the way of setting the warehouse dir is described as follows: * If `spark.sql.warehouse.dir` is set, `hive.metastore.warehouse.dir` will be automatically set to the value of `spark.sql.warehouse.dir`. The warehouse dir is effectively set to the value of `spark.sql.warehouse.dir`. * If `spark.sql.warehouse.dir` is not set but `hive.metastore.warehouse.dir` is set, `spark.sql.warehouse.dir` will be automatically set to the value of `hive.metastore.warehouse.dir`. The warehouse dir is effectively set to the value of `hive.metastore.warehouse.dir`. * If neither `spark.sql.warehouse.dir` nor `hive.metastore.warehouse.dir` is set, `hive.metastore.warehouse.dir` will be automatically set to the default value of `spark.sql.warehouse.dir`. The warehouse dir is effectively set to the default value of `spark.sql.warehouse.dir`. ## How was this patch tested? `set hive.metastore.warehouse.dir` in `HiveSparkSubmitSuite`. JIRA: https://issues.apache.org/jira/browse/SPARK-15959 Author: Yin Huai <yhuai@databricks.com> Closes #13679 from yhuai/hiveWarehouseDir.
* [SPARK-15011][SQL] Re-enable 'analyze MetastoreRelations' in hive ↵Herman van Hovell2016-06-142-5/+10
| | | | | | | | | | | | | | | | | StatisticsSuite ## What changes were proposed in this pull request? This test re-enables the `analyze MetastoreRelations` in `org.apache.spark.sql.hive.StatisticsSuite`. The flakiness of this test was traced back to a shared configuration option, `hive.exec.compress.output`, in `TestHive`. This property was set to `true` by the `HiveCompatibilitySuite`. I have added configuration resetting logic to `HiveComparisonTest`, in order to prevent such a thing from happening again. ## How was this patch tested? Is a test. Author: Herman van Hovell <hvanhovell@databricks.com> Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #13498 from hvanhovell/SPARK-15011.
* [SPARK-15864][SQL] Fix Inconsistent Behaviors when Uncaching Non-cached Tablesgatorsmile2016-06-141-4/+20
| | | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? To uncache a table, we have three different ways: - _SQL interface_: `UNCACHE TABLE` - _DataSet API_: `sparkSession.catalog.uncacheTable` - _DataSet API_: `sparkSession.table(tableName).unpersist()` When the table is not cached, - _SQL interface_: `UNCACHE TABLE non-cachedTable` -> **no error message** - _Dataset API_: `sparkSession.catalog.uncacheTable("non-cachedTable")` -> **report a strange error message:** ```requirement failed: Table [a: int] is not cached``` - _Dataset API_: `sparkSession.table("non-cachedTable").unpersist()` -> **no error message** This PR will make them consistent. No operation if the table has already been uncached. In addition, this PR also removes `uncacheQuery` and renames `tryUncacheQuery` to `uncacheQuery`, and documents it that it's noop if the table has already been uncached #### How was this patch tested? Improved the existing test case for verifying the cases when the table has not been cached. Also added test cases for verifying the cases when the table does not exist Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #13593 from gatorsmile/uncacheNonCachedTable.
* [SPARK-15655][SQL] Fix Wrong Partition Column Order when Fetching ↵gatorsmile2016-06-141-0/+32
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Partitioned Tables #### What changes were proposed in this pull request? When fetching the partitioned table, the output contains wrong results. The order of partition key values do not match the order of partition key columns in output schema. For example, ```SQL CREATE TABLE table_with_partition(c1 string) PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) INSERT OVERWRITE TABLE table_with_partition PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') SELECT 'blarr' SELECT p1, p2, p3, p4, p5, c1 FROM table_with_partition ``` ``` +---+---+---+---+---+-----+ | p1| p2| p3| p4| p5| c1| +---+---+---+---+---+-----+ | d| e| c| b| a|blarr| +---+---+---+---+---+-----+ ``` The expected result should be ``` +---+---+---+---+---+-----+ | p1| p2| p3| p4| p5| c1| +---+---+---+---+---+-----+ | a| b| c| d| e|blarr| +---+---+---+---+---+-----+ ``` This PR is to fix this by enforcing the order matches the table partition definition. #### How was this patch tested? Added a test case into `SQLQuerySuite` Author: gatorsmile <gatorsmile@gmail.com> Closes #13400 from gatorsmile/partitionedTableFetch.
* [SPARK-15663][SQL] SparkSession.catalog.listFunctions shouldn't include the ↵Sandeep Singh2016-06-131-17/+31
| | | | | | | | | | | | | | list of built-in functions ## What changes were proposed in this pull request? SparkSession.catalog.listFunctions currently returns all functions, including the list of built-in functions. This makes the method not as useful because anytime it is run the result set contains over 100 built-in functions. ## How was this patch tested? CatalogSuite Author: Sandeep Singh <sandeep@techaddict.me> Closes #13413 from techaddict/SPARK-15663.
* [SPARK-15808][SQL] File Format Checking When Appending Datagatorsmile2016-06-131-0/+72
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? **Issue:** Got wrong results or strange errors when append data to a table with mismatched file format. _Example 1: PARQUET -> CSV_ ```Scala createDF(0, 9).write.format("parquet").saveAsTable("appendParquetToOrc") createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendParquetToOrc") ``` Error we got: ``` Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 2, localhost): java.lang.RuntimeException: file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-bc8fedf2-aa6a-4002-a18b-524c6ac859d4/appendorctoparquet/part-r-00000-c0e3f365-1d46-4df5-a82c-b47d7af9feb9.snappy.orc is not a Parquet file. expected magic number at tail [80, 65, 82, 49] but found [79, 82, 67, 23] ``` _Example 2: Json -> CSV_ ```Scala createDF(0, 9).write.format("json").saveAsTable("appendJsonToCSV") createDF(10, 19).write.mode(SaveMode.Append).format("parquet").saveAsTable("appendJsonToCSV") ``` No exception, but wrong results: ``` +----+----+ | c1| c2| +----+----+ |null|null| |null|null| |null|null| |null|null| | 0|str0| | 1|str1| | 2|str2| | 3|str3| | 4|str4| | 5|str5| | 6|str6| | 7|str7| | 8|str8| | 9|str9| +----+----+ ``` _Example 3: Json -> Text_ ```Scala createDF(0, 9).write.format("json").saveAsTable("appendJsonToText") createDF(10, 19).write.mode(SaveMode.Append).format("text").saveAsTable("appendJsonToText") ``` Error we got: ``` Text data source supports only a single column, and you have 2 columns. ``` This PR is to issue an exception with appropriate error messages. #### How was this patch tested? Added test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #13546 from gatorsmile/fileFormatCheck.
* [SPARK-15887][SQL] Bring back the hive-site.xml support for Spark 2.0Wenchen Fan2016-06-132-0/+31
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Right now, Spark 2.0 does not load hive-site.xml. Based on users' feedback, it seems make sense to still load this conf file. This PR adds a `hadoopConf` API in `SharedState`, which is `sparkContext.hadoopConfiguration` by default. When users are under hive context, `SharedState.hadoopConf` will load hive-site.xml and append its configs to `sparkContext.hadoopConfiguration`. When we need to read hadoop config in spark sql, we should call `SessionState.newHadoopConf`, which contains `sparkContext.hadoopConfiguration`, hive-site.xml and sql configs. ## How was this patch tested? new test in `HiveDataFrameSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #13611 from cloud-fan/hive-site.
* Revert "[SPARK-15753][SQL] Move Analyzer stuff to Analyzer from DataFrameWriter"Wenchen Fan2016-06-121-2/+2
| | | | This reverts commit 0ec279ffdf92853965e327a9f0f6956cacb7a23e.
* [SPARK-15678] Add support to REFRESH data source pathsSameer Agarwal2016-06-101-0/+45
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Spark currently incorrectly continues to use cached data even if the underlying data is overwritten. Current behavior: ```scala val dir = "/tmp/test" sqlContext.range(1000).write.mode("overwrite").parquet(dir) val df = sqlContext.read.parquet(dir).cache() df.count() // outputs 1000 sqlContext.range(10).write.mode("overwrite").parquet(dir) sqlContext.read.parquet(dir).count() // outputs 1000 <---- We are still using the cached dataset ``` This patch fixes this bug by adding support for `REFRESH path` that invalidates and refreshes all the cached data (and the associated metadata) for any dataframe that contains the given data source path. Expected behavior: ```scala val dir = "/tmp/test" sqlContext.range(1000).write.mode("overwrite").parquet(dir) val df = sqlContext.read.parquet(dir).cache() df.count() // outputs 1000 sqlContext.range(10).write.mode("overwrite").parquet(dir) spark.catalog.refreshResource(dir) sqlContext.read.parquet(dir).count() // outputs 10 <---- We are not using the cached dataset ``` ## How was this patch tested? Unit tests for overwrites and appends in `ParquetQuerySuite` and `CachedTableSuite`. Author: Sameer Agarwal <sameer@databricks.com> Closes #13566 from sameeragarwal/refresh-path-2.
* [SPARK-15654] [SQL] fix non-splitable files for text based file formatsDavies Liu2016-06-101-1/+1
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, we always split the files when it's bigger than maxSplitBytes, but Hadoop LineRecordReader does not respect the splits for compressed files correctly, we should have a API for FileFormat to check whether the file could be splitted or not. This PR is based on #13442, closes #13442 ## How was this patch tested? add regression tests. Author: Davies Liu <davies@databricks.com> Closes #13531 from davies/fix_split.
* [MINOR][X][X] Replace all occurrences of None: Option with Option.emptySandeep Singh2016-06-101-5/+5
| | | | | | | | | | | | ## What changes were proposed in this pull request? Replace all occurrences of `None: Option[X]` with `Option.empty[X]` ## How was this patch tested? Exisiting Tests Author: Sandeep Singh <sandeep@techaddict.me> Closes #13591 from techaddict/minor-7.
* [SPARK-15871][SQL] Add `assertNotPartitioned` check in `DataFrameWriter`Liwei Lin2016-06-101-4/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? It doesn't make sense to specify partitioning parameters, when we write data out from Datasets/DataFrames into `jdbc` tables or streaming `ForeachWriter`s. This patch adds `assertNotPartitioned` check in `DataFrameWriter`. <table> <tr> <td align="center"><strong>operation</strong></td> <td align="center"><strong>should check not partitioned?</strong></td> </tr> <tr> <td align="center">mode</td> <td align="center"></td> </tr> <tr> <td align="center">outputMode</td> <td align="center"></td> </tr> <tr> <td align="center">trigger</td> <td align="center"></td> </tr> <tr> <td align="center">format</td> <td align="center"></td> </tr> <tr> <td align="center">option/options</td> <td align="center"></td> </tr> <tr> <td align="center">partitionBy</td> <td align="center"></td> </tr> <tr> <td align="center">bucketBy</td> <td align="center"></td> </tr> <tr> <td align="center">sortBy</td> <td align="center"></td> </tr> <tr> <td align="center">save</td> <td align="center"></td> </tr> <tr> <td align="center">queryName</td> <td align="center"></td> </tr> <tr> <td align="center">startStream</td> <td align="center"></td> </tr> <tr> <td align="center">foreach</td> <td align="center">yes</td> </tr> <tr> <td align="center">insertInto</td> <td align="center"></td> </tr> <tr> <td align="center">saveAsTable</td> <td align="center"></td> </tr> <tr> <td align="center">jdbc</td> <td align="center">yes</td> </tr> <tr> <td align="center">json</td> <td align="center"></td> </tr> <tr> <td align="center">parquet</td> <td align="center"></td> </tr> <tr> <td align="center">orc</td> <td align="center"></td> </tr> <tr> <td align="center">text</td> <td align="center"></td> </tr> <tr> <td align="center">csv</td> <td align="center"></td> </tr> </table> ## How was this patch tested? New dedicated tests. Author: Liwei Lin <lwlin7@gmail.com> Closes #13597 from lw-lin/add-assertNotPartitioned.