aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-17346][SQL] Add Kafka source for Structured StreamingShixiong Zhu2016-10-051-10/+30
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source. It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing tdas did most of work and part of them was inspired by koeninger's work. ### Introduction The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows: Column | Type ---- | ---- key | binary value | binary topic | string partition | int offset | long timestamp | long timestampType | int The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic. ### Configuration The user can use `DataStreamReader.option` to set the following configurations. Kafka Source's options | value | default | meaning ------ | ------- | ------ | ----- startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off. failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected. subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source. subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source. kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets. fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")` ### Usage * Subscribe to 1 topic ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribe", "topic1") .load() ``` * Subscribe to multiple topics ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribe", "topic1,topic2") .load() ``` * Subscribe to a pattern ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribePattern", "topic.*") .load() ``` ## How was this patch tested? The new unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Shixiong Zhu <zsxwing@gmail.com> Author: cody koeninger <cody@koeninger.org> Closes #15102 from zsxwing/kafka-source.
* [SPARK-17328][SQL] Fix NPE with EXPLAIN DESCRIBE TABLEDongjoon Hyun2016-10-053-16/+64
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes the following NPE scenario in two ways. **Reported Error Scenario** ```scala scala> sql("EXPLAIN DESCRIBE TABLE x").show(truncate = false) INFO SparkSqlParser: Parsing command: EXPLAIN DESCRIBE TABLE x java.lang.NullPointerException ``` - **DESCRIBE**: Extend `DESCRIBE` syntax to accept `TABLE`. - **EXPLAIN**: Prevent NPE in case of the parsing failure of target statement, e.g., `EXPLAIN DESCRIBE TABLES x`. ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #15357 from dongjoon-hyun/SPARK-17328.
* [SPARK-17258][SQL] Parse scientific decimal literals as decimalsHerman van Hovell2016-10-043-12/+22
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently Spark SQL parses regular decimal literals (e.g. `10.00`) as decimals and scientific decimal literals (e.g. `10.0e10`) as doubles. The difference between the two confuses most users. This PR unifies the parsing behavior and also parses scientific decimal literals as decimals. This implications in tests are limited to a single Hive compatibility test. ## How was this patch tested? Updated tests in `ExpressionParserSuite` and `SQLQueryTestSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14828 from hvanhovell/SPARK-17258.
* [SPARK-17549][SQL] Only collect table size stat in driver for cached relation.Marcelo Vanzin2016-10-041-0/+14
| | | | | | | | | | | | | | | | | | | | | | | | This reverts commit 9ac68dbc5720026ea92acc61d295ca64d0d3d132. Turns out the original fix was correct. Original change description: The existing code caches all stats for all columns for each partition in the driver; for a large relation, this causes extreme memory usage, which leads to gc hell and application failures. It seems that only the size in bytes of the data is actually used in the driver, so instead just colllect that. In executors, the full stats are still kept, but that's not a big problem; we expect the data to be distributed and thus not really incur in too much memory pressure in each individual executor. There are also potential improvements on the executor side, since the data being stored currently is very wasteful (e.g. storing boxed types vs. primitive types for stats). But that's a separate issue. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #15304 from vanzin/SPARK-17549.2.
* [SPARK-17073][SQL] generate column-level statisticsZhenhua Wang2016-10-033-14/+465
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Generate basic column statistics for all the atomic types: - numeric types: max, min, num of nulls, ndv (number of distinct values) - date/timestamp types: they are also represented as numbers internally, so they have the same stats as above. - string: avg length, max length, num of nulls, ndv - binary: avg length, max length, num of nulls - boolean: num of nulls, num of trues, num of falsies Also support storing and loading these statistics. One thing to notice: We support analyzing columns independently, e.g.: sql1: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key;` sql2: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS value;` when running sql2 to collect column stats for `value`, we don’t remove stats of columns `key` which are analyzed in sql1 and not in sql2. As a result, **users need to guarantee consistency** between sql1 and sql2. If the table has been changed before sql2, users should re-analyze column `key` when they want to analyze column `value`: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key, value;` ## How was this patch tested? add unit tests Author: Zhenhua Wang <wzh_zju@163.com> Closes #15090 from wzhfy/colStats.
* [SPARK-14914][CORE][SQL] Skip/fix some test cases on Windows due to ↵Tao LI2016-10-021-1/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | limitation of Windows ## What changes were proposed in this pull request? This PR proposes to fix/skip some tests failed on Windows. This PR takes over https://github.com/apache/spark/pull/12696. **Before** - **SparkSubmitSuite** ``` [info] - launch simple application with spark-submit *** FAILED *** (202 milliseconds) [info] java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specifie [info] - includes jars passed in through --jars *** FAILED *** (1 second, 625 milliseconds) [info] java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified ``` - **DiskStoreSuite** ``` [info] - reads of memory-mapped and non memory-mapped files are equivalent *** FAILED *** (1 second, 78 milliseconds) [info] diskStoreMapped.remove(blockId) was false (DiskStoreSuite.scala:41) ``` **After** - **SparkSubmitSuite** ``` [info] - launch simple application with spark-submit (578 milliseconds) [info] - includes jars passed in through --jars (1 second, 875 milliseconds) ``` - **DiskStoreSuite** ``` [info] DiskStoreSuite: [info] - reads of memory-mapped and non memory-mapped files are equivalent !!! CANCELED !!! (766 milliseconds ``` For `CreateTableAsSelectSuite` and `FsHistoryProviderSuite`, I could not reproduce as the Java version seems higher than the one that has the bugs about `setReadable(..)` and `setWritable(...)` but as they are bugs reported clearly, it'd be sensible to skip those. We should revert the changes for both back as soon as we drop the support of Java 7. ## How was this patch tested? Manually tested via AppVeyor. Closes #12696 Author: Tao LI <tl@microsoft.com> Author: U-FAREAST\tl <tl@microsoft.com> Author: hyukjinkwon <gurwls223@gmail.com> Closes #15320 from HyukjinKwon/SPARK-14914.
* [SPARK-17717][SQL] Add Exist/find methods to Catalog [FOLLOW-UP]Herman van Hovell2016-10-011-19/+19
| | | | | | | | | | | | ## What changes were proposed in this pull request? We added find and exists methods for Databases, Tables and Functions to the user facing Catalog in PR https://github.com/apache/spark/pull/15301. However, it was brought up that the semantics of the `find` methods are more in line a `get` method (get an object or else fail). So we rename these in this PR. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15308 from hvanhovell/SPARK-17717-2.
* [SPARK-17740] Spark tests should mock / interpose HDFS to ensure that ↵Eric Liang2016-09-303-4/+19
| | | | | | | | | | | | | | | | | | | streams are closed ## What changes were proposed in this pull request? As a followup to SPARK-17666, ensure filesystem connections are not leaked at least in unit tests. This is done here by intercepting filesystem calls as suggested by JoshRosen . At the end of each test, we assert no filesystem streams are left open. This applies to all tests using SharedSQLContext or SharedSparkContext. ## How was this patch tested? I verified that tests in sql and core are indeed using the filesystem backend, and fixed the detected leaks. I also checked that reverting https://github.com/apache/spark/pull/15245 causes many actual test failures due to connection leaks. Author: Eric Liang <ekl@databricks.com> Author: Eric Liang <ekhliang@gmail.com> Closes #15306 from ericl/sc-4672.
* [SPARK-17738] [SQL] fix ARRAY/MAP in columnar cacheDavies Liu2016-09-301-3/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The actualSize() of array and map is different from the actual size, the header is Int, rather than Long. ## How was this patch tested? The flaky test should be fixed. Author: Davies Liu <davies@databricks.com> Closes #15305 from davies/fix_MAP.
* [SPARK-17717][SQL] Add exist/find methods to Catalog.Herman van Hovell2016-09-291-0/+118
| | | | | | | | | | | | ## What changes were proposed in this pull request? The current user facing catalog does not implement methods for checking object existence or finding objects. You could theoretically do this using the `list*` commands, but this is rather cumbersome and can actually be costly when there are many objects. This PR adds `exists*` and `find*` methods for Databases, Table and Functions. ## How was this patch tested? Added tests to `org.apache.spark.sql.internal.CatalogSuite` Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15301 from hvanhovell/SPARK-17717.
* [SPARK-17612][SQL] Support `DESCRIBE table PARTITION` SQL syntaxDongjoon Hyun2016-09-292-0/+117
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `DESCRIBE table PARTITION` SQL Syntax again. It was supported until Spark 1.6.2, but was dropped since 2.0.0. **Spark 1.6.2** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res1: org.apache.spark.sql.DataFrame = [result: string] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res2: org.apache.spark.sql.DataFrame = [result: string] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) +----------------------------------------------------------------+ |result | +----------------------------------------------------------------+ |a string | |b int | |c string | |d string | | | |# Partition Information | |# col_name data_type comment | | | |c string | |d string | +----------------------------------------------------------------+ ``` **Spark 2.0** - **Before** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res1: org.apache.spark.sql.DataFrame = [] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) org.apache.spark.sql.catalyst.parser.ParseException: Unsupported SQL statement ``` - **After** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res1: org.apache.spark.sql.DataFrame = [] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) +-----------------------+---------+-------+ |col_name |data_type|comment| +-----------------------+---------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information| | | |# col_name |data_type|comment| |c |string |null | |d |string |null | +-----------------------+---------+-------+ scala> sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)").show(100,false) +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ |col_name |data_type|comment| +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information | | | |# col_name |data_type|comment| |c |string |null | |d |string |null | | | | | |Detailed Partition Information CatalogPartition( Partition Values: [Us, 1] Storage(Location: file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1]) Partition Parameters:{transient_lastDdlTime=1475001066})| | | +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ scala> sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)").show(100,false) +--------------------------------+---------------------------------------------------------------------------------------+-------+ |col_name |data_type |comment| +--------------------------------+---------------------------------------------------------------------------------------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information | | | |# col_name |data_type |comment| |c |string |null | |d |string |null | | | | | |# Detailed Partition Information| | | |Partition Value: |[Us, 1] | | |Database: |default | | |Table: |partitioned_table | | |Location: |file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1| | |Partition Parameters: | | | | transient_lastDdlTime |1475001066 | | | | | | |# Storage Information | | | |SerDe Library: |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe | | |InputFormat: |org.apache.hadoop.mapred.TextInputFormat | | |OutputFormat: |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat | | |Compressed: |No | | |Storage Desc Parameters: | | | | serialization.format |1 | | +--------------------------------+---------------------------------------------------------------------------------------+-------+ ``` ## How was this patch tested? Pass the Jenkins tests with a new testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #15168 from dongjoon-hyun/SPARK-17612.
* [SPARK-17699] Support for parsing JSON string columnsMichael Armbrust2016-09-293-1/+32
| | | | | | | | | | | | | | | | | | Spark SQL has great support for reading text files that contain JSON data. However, in many cases the JSON data is just one column amongst others. This is particularly true when reading from sources such as Kafka. This PR adds a new functions `from_json` that converts a string column into a nested `StructType` with a user specified schema. Example usage: ```scala val df = Seq("""{"a": 1}""").toDS() val schema = new StructType().add("a", IntegerType) df.select(from_json($"value", schema) as 'json) // => [json: <a: int>] ``` This PR adds support for java, scala and python. I leveraged our existing JSON parsing support by moving it into catalyst (so that we could define expressions using it). I left SQL out for now, because I'm not sure how users would specify a schema. Author: Michael Armbrust <michael@databricks.com> Closes #15274 from marmbrus/jsonParser.
* [SPARK-17641][SQL] Collect_list/Collect_set should not collect null values.Herman van Hovell2016-09-281-0/+12
| | | | | | | | | | | | ## What changes were proposed in this pull request? We added native versions of `collect_set` and `collect_list` in Spark 2.0. These currently also (try to) collect null values, this is different from the original Hive implementation. This PR fixes this by adding a null check to the `Collect.update` method. ## How was this patch tested? Added a regression test to `DataFrameAggregateSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15208 from hvanhovell/SPARK-17641.
* [SPARK-17713][SQL] Move row-datasource related tests out of JDBCSuiteEric Liang2016-09-282-8/+72
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? As a followup for https://github.com/apache/spark/pull/15273 we should move non-JDBC specific tests out of that suite. ## How was this patch tested? Ran the test. Author: Eric Liang <ekl@databricks.com> Closes #15287 from ericl/spark-17713.
* [SPARK-17673][SQL] Incorrect exchange reuse with RowDataSourceScanEric Liang2016-09-281-0/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? It seems the equality check for reuse of `RowDataSourceScanExec` nodes doesn't respect the output schema. This can cause self-joins or unions over the same underlying data source to return incorrect results if they select different fields. ## How was this patch tested? New unit test passes after the fix. Author: Eric Liang <ekl@databricks.com> Closes #15273 from ericl/spark-17673.
* [SPARK-15962][SQL] Introduce implementation with a dense format for ↵Kazuaki Ishizaki2016-09-272-2/+234
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | UnsafeArrayData ## What changes were proposed in this pull request? This PR introduces more compact representation for ```UnsafeArrayData```. ```UnsafeArrayData``` needs to accept ```null``` value in each entry of an array. In the current version, it has three parts ``` [numElements] [offsets] [values] ``` `Offsets` has the number of `numElements`, and represents `null` if its value is negative. It may increase memory footprint, and introduces an indirection for accessing each of `values`. This PR uses bitvectors to represent nullability for each element like `UnsafeRow`, and eliminates an indirection for accessing each element. The new ```UnsafeArrayData``` has four parts. ``` [numElements][null bits][values or offset&length][variable length portion] ``` In the `null bits` region, we store 1 bit per element, represents whether an element is null. Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries. In the `values or offset&length` region, we store the content of elements. For fields that hold fixed-length primitive types, such as long, double, or int, we store the value directly in the field. For fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the base address of the array) that points to the beginning of the variable-length field and length (they are combined into a long). Each is word-aligned. For `variable length portion`, each is aligned to 8-byte boundaries. The new format can reduce memory footprint and improve performance of accessing each element. An example of memory foot comparison: 1024x1024 elements integer array Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024 + 1024x1024 = 2M bytes Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024/8 + 1024x1024 = 1.25M bytes In summary, we got 1.0-2.6x performance improvements over the code before applying this PR. Here are performance results of [benchmark programs](https://github.com/kiszk/spark/blob/04d2e4b6dbdc4eff43ce18b3c9b776e0129257c7/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala): **Read UnsafeArrayData**: 1.7x and 1.6x performance improvements over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 430 / 436 390.0 2.6 1.0X Double 456 / 485 367.8 2.7 0.9X With SPARK-15962 Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 252 / 260 666.1 1.5 1.0X Double 281 / 292 597.7 1.7 0.9X ```` **Write UnsafeArrayData**: 1.0x and 1.1x performance improvements over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 203 / 273 103.4 9.7 1.0X Double 239 / 356 87.9 11.4 0.8X With SPARK-15962 Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 196 / 249 107.0 9.3 1.0X Double 227 / 367 92.3 10.8 0.9X ```` **Get primitive array from UnsafeArrayData**: 2.6x and 1.6x performance improvements over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 207 / 217 304.2 3.3 1.0X Double 257 / 363 245.2 4.1 0.8X With SPARK-15962 Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 151 / 198 415.8 2.4 1.0X Double 214 / 394 293.6 3.4 0.7X ```` **Create UnsafeArrayData from primitive array**: 1.7x and 2.1x performance improvements over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 340 / 385 185.1 5.4 1.0X Double 479 / 705 131.3 7.6 0.7X With SPARK-15962 Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Int 206 / 211 306.0 3.3 1.0X Double 232 / 406 271.6 3.7 0.9X ```` 1.7x and 1.4x performance improvements in [```UDTSerializationBenchmark```](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala) over the code before applying this PR ```` OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 Intel Xeon E3-12xx v2 (Ivy Bridge) Without SPARK-15962 VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ serialize 442 / 533 0.0 441927.1 1.0X deserialize 217 / 274 0.0 217087.6 2.0X With SPARK-15962 VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ serialize 265 / 318 0.0 265138.5 1.0X deserialize 155 / 197 0.0 154611.4 1.7X ```` ## How was this patch tested? Added unit tests into ```UnsafeArraySuite``` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #13680 from kiszk/SPARK-15962.
* [SPARK-17652] Fix confusing exception message while reserving capacitySameer Agarwal2016-09-261-2/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This minor patch fixes a confusing exception message while reserving additional capacity in the vectorized parquet reader. ## How was this patch tested? Exisiting Unit Tests Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #15225 from sameeragarwal/error-msg.
* [SPARK-17153][SQL] Should read partition data when reading new files in ↵Liang-Chi Hsieh2016-09-262-1/+90
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | filestream without globbing ## What changes were proposed in this pull request? When reading file stream with non-globbing path, the results return data with all `null`s for the partitioned columns. E.g., case class A(id: Int, value: Int) val data = spark.createDataset(Seq( A(1, 1), A(2, 2), A(2, 3)) ) val url = "/tmp/test" data.write.partitionBy("id").parquet(url) spark.read.parquet(url).show +-----+---+ |value| id| +-----+---+ | 2| 2| | 3| 2| | 1| 1| +-----+---+ val s = spark.readStream.schema(spark.read.load(url).schema).parquet(url) s.writeStream.queryName("test").format("memory").start() sql("SELECT * FROM test").show +-----+----+ |value| id| +-----+----+ | 2|null| | 3|null| | 1|null| +-----+----+ ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #14803 from viirya/filestreamsource-option.
* [SPARK-14525][SQL] Make DataFrameWrite.save work for jdbcJustin Pihony2016-09-261-0/+82
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This change modifies the implementation of DataFrameWriter.save such that it works with jdbc, and the call to jdbc merely delegates to save. ## How was this patch tested? This was tested via unit tests in the JDBCWriteSuite, of which I added one new test to cover this scenario. ## Additional details rxin This seems to have been most recently touched by you and was also commented on in the JIRA. This contribution is my original work and I license the work to the project under the project's open source license. Author: Justin Pihony <justin.pihony@gmail.com> Author: Justin Pihony <justin.pihony@typesafe.com> Closes #12601 from JustinPihony/jdbc_reconciliation.
* [SPARK-17551][SQL] Add DataFrame API for null orderingxin wu2016-09-253-0/+18
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request adds Scala/Java DataFrame API for null ordering (NULLS FIRST | LAST). Also did some minor clean up for related code (e.g. incorrect indentation), and renamed "orderby-nulls-ordering.sql" to be consistent with existing test files. ## How was this patch tested? Added a new test case in DataFrameSuite. Author: petermaxlee <petermaxlee@gmail.com> Author: Xin Wu <xinwu@us.ibm.com> Closes #15123 from petermaxlee/SPARK-17551.
* [SPARK-17643] Remove comparable requirement from OffsetMichael Armbrust2016-09-231-39/+0
| | | | | | | | For some sources, it is difficult to provide a global ordering based only on the data in the offset. Since we don't use comparison for correctness, lets remove it. Author: Michael Armbrust <michael@databricks.com> Closes #15207 from marmbrus/removeComparable.
* [SPARK-17640][SQL] Avoid using -1 as the default batchId for ↵Shixiong Zhu2016-09-221-12/+12
| | | | | | | | | | | | | | | | FileStreamSource.FileEntry ## What changes were proposed in this pull request? Avoid using -1 as the default batchId for FileStreamSource.FileEntry so that we can make sure not writing any FileEntry(..., batchId = -1) into the log. This also avoids people misusing it in future (#15203 is an example). ## How was this patch tested? Jenkins. Author: Shixiong Zhu <shixiong@databricks.com> Closes #15206 from zsxwing/cleanup.
* [SPARK-17569][SPARK-17569][TEST] Make the unit test added for work againBurak Yavuz2016-09-221-1/+1
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? A [PR](https://github.com/apache/spark/commit/a6aade0042d9c065669f46d2dac40ec6ce361e63) was merged concurrently that made the unit test for PR #15122 not test anything anymore. This PR fixes the test. ## How was this patch tested? Changed line https://github.com/apache/spark/blob/0d634875026ccf1eaf984996e9460d7673561f80/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L137 from `false` to `true` and made sure the unit test failed. Author: Burak Yavuz <brkyvz@gmail.com> Closes #15203 from brkyvz/fix-test.
* [SPARK-17616][SQL] Support a single distinct aggregate combined with a ↵Herman van Hovell2016-09-221-0/+8
| | | | | | | | | | | | | | | | | | | | | | non-partial aggregate ## What changes were proposed in this pull request? We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example: ```sql select grp, collect_list(col1), count(distinct col2) from tbl_a group by 1 ``` This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6). ## How was this patch tested? Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15187 from hvanhovell/SPARK-17616.
* [SPARK-17613] S3A base paths with no '/' at the end return empty DataFramesBurak Yavuz2016-09-221-1/+44
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Consider you have a bucket as `s3a://some-bucket` and under it you have files: ``` s3a://some-bucket/file1.parquet s3a://some-bucket/file2.parquet ``` Getting the parent path of `s3a://some-bucket/file1.parquet` yields `s3a://some-bucket/` and the ListingFileCatalog uses this as the key in the hash map. When catalog.allFiles is called, we use `s3a://some-bucket` (no slash at the end) to get the list of files, and we're left with an empty list! This PR fixes this by adding a `/` at the end of the `URI` iff the given `Path` doesn't have a parent, i.e. is the root. This is a no-op if the path already had a `/` at the end, and is handled through the Hadoop Path, path merging semantics. ## How was this patch tested? Unit test in `FileCatalogSuite`. Author: Burak Yavuz <brkyvz@gmail.com> Closes #15169 from brkyvz/SPARK-17613.
* [SPARK-17625][SQL] set expectedOutputAttributes when converting ↵Zhenhua Wang2016-09-221-1/+13
| | | | | | | | | | | | | | | | SimpleCatalogRelation to LogicalRelation ## What changes were proposed in this pull request? We should set expectedOutputAttributes when converting SimpleCatalogRelation to LogicalRelation, otherwise the outputs of LogicalRelation are different from outputs of SimpleCatalogRelation - they have different exprId's. ## How was this patch tested? add a test case Author: Zhenhua Wang <wzh_zju@163.com> Closes #15182 from wzhfy/expectedAttributes.
* [SPARK-17492][SQL] Fix Reading Cataloged Data Sources without Extending ↵gatorsmile2016-09-223-22/+95
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | SchemaRelationProvider ### What changes were proposed in this pull request? For data sources without extending `SchemaRelationProvider`, we expect users to not specify schemas when they creating tables. If the schema is input from users, an exception is issued. Since Spark 2.1, for any data source, to avoid infer the schema every time, we store the schema in the metastore catalog. Thus, when reading a cataloged data source table, the schema could be read from metastore catalog. In this case, we also got an exception. For example, ```Scala sql( s""" |CREATE TABLE relationProvierWithSchema |USING org.apache.spark.sql.sources.SimpleScanSource |OPTIONS ( | From '1', | To '10' |) """.stripMargin) spark.table(tableName).show() ``` ``` org.apache.spark.sql.sources.SimpleScanSource does not allow user-specified schemas.; ``` This PR is to fix the above issue. When building a data source, we introduce a flag `isSchemaFromUsers` to indicate whether the schema is really input from users. If true, we issue an exception. Otherwise, we will call the `createRelation` of `RelationProvider` to generate the `BaseRelation`, in which it contains the actual schema. ### How was this patch tested? Added a few cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #15046 from gatorsmile/tempViewCases.
* [SPARK-17569] Make StructuredStreaming FileStreamSource batch generation fasterBurak Yavuz2016-09-211-1/+52
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? While getting the batch for a `FileStreamSource` in StructuredStreaming, we know which files we must take specifically. We already have verified that they exist, and have committed them to a metadata log. When creating the FileSourceRelation however for an incremental execution, the code checks the existence of every single file once again! When you have 100,000s of files in a folder, creating the first batch takes 2 hours+ when working with S3! This PR disables that check ## How was this patch tested? Added a unit test to `FileStreamSource`. Author: Burak Yavuz <brkyvz@gmail.com> Closes #15122 from brkyvz/SPARK-17569.
* [SPARK-17590][SQL] Analyze CTE definitions at once and allow CTE subquery to ↵Liang-Chi Hsieh2016-09-211-0/+25
| | | | | | | | | | | | | | | | | | define CTE ## What changes were proposed in this pull request? We substitute logical plan with CTE definitions in the analyzer rule CTESubstitution. A CTE definition can be used in the logical plan for multiple times, and its analyzed logical plan should be the same. We should not analyze CTE definitions multiple times when they are reused in the query. By analyzing CTE definitions before substitution, we can support defining CTE in subquery. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #15146 from viirya/cte-analysis-once.
* [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't existBurak Yavuz2016-09-211-0/+11
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The `ListingFileCatalog` lists files given a set of resolved paths. If a folder is deleted at any time between the paths were resolved and the file catalog can check for the folder, the Spark job fails. This may abruptly stop long running StructuredStreaming jobs for example. Folders may be deleted by users or automatically by retention policies. These cases should not prevent jobs from successfully completing. ## How was this patch tested? Unit test in `FileCatalogSuite` Author: Burak Yavuz <brkyvz@gmail.com> Closes #15153 from brkyvz/SPARK-17599.
* [SPARK-17513][SQL] Make StreamExecution garbage-collect its metadatapetermaxlee2016-09-201-0/+24
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR modifies StreamExecution such that it discards metadata for batches that have already been fully processed. I used the purge method that was added as part of SPARK-17235. This is a resubmission of 15126, which was based on work by frreiss in #15067, but fixed the test case along with some typos. ## How was this patch tested? A new test case in StreamingQuerySuite. The test case would fail without the changes in this pull request. Author: petermaxlee <petermaxlee@gmail.com> Closes #15166 from petermaxlee/SPARK-17513-2.
* [SPARK-17549][SQL] Revert "[] Only collect table size stat in driver for ↵Yin Huai2016-09-201-14/+0
| | | | | | | | | | cached relation." This reverts commit 39e2bad6a866d27c3ca594d15e574a1da3ee84cc because of the problem mentioned at https://issues.apache.org/jira/browse/SPARK-17549?focusedCommentId=15505060&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15505060 Author: Yin Huai <yhuai@databricks.com> Closes #15157 from yhuai/revert-SPARK-17549.
* [SPARK-15698][SQL][STREAMING] Add the ability to remove the old MetadataLog ↵jerryshao2016-09-202-18/+116
| | | | | | | | | | | | | | | | in FileStreamSource ## What changes were proposed in this pull request? Current `metadataLog` in `FileStreamSource` will add a checkpoint file in each batch but do not have the ability to remove/compact, which will lead to large number of small files when running for a long time. So here propose to compact the old logs into one file. This method is quite similar to `FileStreamSinkLog` but simpler. ## How was this patch tested? Unit test added. Author: jerryshao <sshao@hortonworks.com> Closes #13513 from jerryshao/SPARK-15698.
* [SPARK-17502][SQL] Fix Multiple Bugs in DDL Statements on Temporary Viewsgatorsmile2016-09-201-1/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? - When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for partition-related ALTER TABLE commands. However, it always reports a confusing error message. For example, ``` Partition spec is invalid. The spec (a, b) must match the partition spec () defined in table '`testview`'; ``` - When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for `ALTER TABLE ... UNSET TBLPROPERTIES`. However, it reports a missing table property. For example, ``` Attempted to unset non-existent property 'p' in table '`testView`'; ``` - When `ANALYZE TABLE` is called on a view or a temporary view, we should issue an error message. However, it reports a strange error: ``` ANALYZE TABLE is not supported for Project ``` - When inserting into a temporary view that is generated from `Range`, we will get the following error message: ``` assertion failed: No plan for 'InsertIntoTable Range (0, 10, step=1, splits=Some(1)), false, false +- Project [1 AS 1#20] +- OneRowRelation$ ``` This PR is to fix the above four issues. ### How was this patch tested? Added multiple test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #15054 from gatorsmile/tempViewDDL.
* Revert "[SPARK-17513][SQL] Make StreamExecution garbage-collect its metadata"Wenchen Fan2016-09-201-24/+0
| | | | This reverts commit be9d57fc9d8b10e4234c01c06ed43fd7dd12c07b.
* [SPARK-17513][SQL] Make StreamExecution garbage-collect its metadatapetermaxlee2016-09-191-0/+24
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR modifies StreamExecution such that it discards metadata for batches that have already been fully processed. I used the purge method that was added as part of SPARK-17235. This is based on work by frreiss in #15067, but fixed the test case along with some typos. ## How was this patch tested? A new test case in StreamingQuerySuite. The test case would fail without the changes in this pull request. Author: petermaxlee <petermaxlee@gmail.com> Author: frreiss <frreiss@us.ibm.com> Closes #15126 from petermaxlee/SPARK-17513.
* [SPARK-17571][SQL] AssertOnQuery.condition should always return Boolean valuepetermaxlee2016-09-183-4/+10
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? AssertOnQuery has two apply constructor: one that accepts a closure that returns boolean, and another that accepts a closure that returns Unit. This is actually very confusing because developers could mistakenly think that AssertOnQuery always require a boolean return type and verifies the return result, when indeed the value of the last statement is ignored in one of the constructors. This pull request makes the two constructor consistent and always require boolean value. It will overall make the test suites more robust against developer errors. As an evidence for the confusing behavior, this change also identified a bug with an existing test case due to file system time granularity. This pull request fixes that test case as well. ## How was this patch tested? This is a test only change. Author: petermaxlee <petermaxlee@gmail.com> Closes #15127 from petermaxlee/SPARK-17571.
* [SPARK-16462][SPARK-16460][SPARK-15144][SQL] Make CSV cast null values properlyLiwei Lin2016-09-182-21/+35
| | | | | | | | | | | | | | | | | | | | ## Problem CSV in Spark 2.0.0: - does not read null values back correctly for certain data types such as `Boolean`, `TimestampType`, `DateType` -- this is a regression comparing to 1.6; - does not read empty values (specified by `options.nullValue`) as `null`s for `StringType` -- this is compatible with 1.6 but leads to problems like SPARK-16903. ## What changes were proposed in this pull request? This patch makes changes to read all empty values back as `null`s. ## How was this patch tested? New test cases. Author: Liwei Lin <lwlin7@gmail.com> Closes #14118 from lw-lin/csv-cast-null.
* [SPARK-17541][SQL] fix some DDL bugs about table management when same-name ↵Wenchen Fan2016-09-183-0/+98
| | | | | | | | | | | | | | | | | | | | | | temp view exists ## What changes were proposed in this pull request? In `SessionCatalog`, we have several operations(`tableExists`, `dropTable`, `loopupRelation`, etc) that handle both temp views and metastore tables/views. This brings some bugs to DDL commands that want to handle temp view only or metastore table/view only. These bugs are: 1. `CREATE TABLE USING` will fail if a same-name temp view exists 2. `Catalog.dropTempView`will un-cache and drop metastore table if a same-name table exists 3. `saveAsTable` will fail or have unexpected behaviour if a same-name temp view exists. These bug fixes are pulled out from https://github.com/apache/spark/pull/14962 and targets both master and 2.0 branch ## How was this patch tested? new regression tests Author: Wenchen Fan <wenchen@databricks.com> Closes #15099 from cloud-fan/fix-view.
* [SPARK-17518][SQL] Block Users to Specify the Internal Data Source Provider Hivegatorsmile2016-09-181-0/+7
| | | | | | | | | | | | ### What changes were proposed in this pull request? In Spark 2.1, we introduced a new internal provider `hive` for telling Hive serde tables from data source tables. This PR is to block users to specify this in `DataFrameWriter` and SQL APIs. ### How was this patch tested? Added a test case Author: gatorsmile <gatorsmile@gmail.com> Closes #15073 from gatorsmile/formatHive.
* Correct fetchsize property name in docsDaniel Darabos2016-09-171-3/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Replace `fetchSize` with `fetchsize` in the docs. ## How was this patch tested? I manually tested `fetchSize` and `fetchsize`. The latter has an effect. See also [`JdbcUtils.scala#L38`](https://github.com/apache/spark/blob/v2.0.0/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala#L38) for the definition of the property. Author: Daniel Darabos <darabos.daniel@gmail.com> Closes #14975 from darabos/patch-3.
* [SPARK-17549][SQL] Only collect table size stat in driver for cached relation.Marcelo Vanzin2016-09-161-0/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | The existing code caches all stats for all columns for each partition in the driver; for a large relation, this causes extreme memory usage, which leads to gc hell and application failures. It seems that only the size in bytes of the data is actually used in the driver, so instead just colllect that. In executors, the full stats are still kept, but that's not a big problem; we expect the data to be distributed and thus not really incur in too much memory pressure in each individual executor. There are also potential improvements on the executor side, since the data being stored currently is very wasteful (e.g. storing boxed types vs. primitive types for stats). But that's a separate issue. On a mildly related change, I'm also adding code to catch exceptions in the code generator since Janino was breaking with the test data I tried this patch on. Tested with unit tests and by doing a count a very wide table (20k columns) with many partitions. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #15112 from vanzin/SPARK-17549.
* [SPARK-17426][SQL] Refactor `TreeNode.toJSON` to avoid OOM when converting ↵Sean Zhong2016-09-161-136/+0
| | | | | | | | | | | | | | | | | | | | | | | unknown fields to JSON ## What changes were proposed in this pull request? This PR is a follow up of SPARK-17356. Current implementation of `TreeNode.toJSON` recursively converts all fields of TreeNode to JSON, even if the field is of type `Seq` or type Map. This may trigger out of memory exception in cases like: 1. the Seq or Map can be very big. Converting them to JSON may take huge memory, which may trigger out of memory error. 2. Some user space input may also be propagated to the Plan. The user space input can be of arbitrary type, and may also be self-referencing. Trying to print user space input to JSON may trigger out of memory error or stack overflow error. For a code example, please check the Jira description of SPARK-17426. In this PR, we refactor the `TreeNode.toJSON` so that we only convert a field to JSON string if the field is a safe type. ## How was this patch tested? Unit test. Author: Sean Zhong <seanzhong@databricks.com> Closes #14990 from clockfly/json_oom2.
* [SPARK-17458][SQL] Alias specified for aggregates in a pivot are not honoredAndrew Ray2016-09-151-0/+11
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This change preserves aliases that are given for pivot aggregations ## How was this patch tested? New unit test Author: Andrew Ray <ray.andrew@gmail.com> Closes #15111 from aray/SPARK-17458.
* [SPARK-17429][SQL] use ImplicitCastInputTypes with function Length岑玉海2016-09-151-4/+6
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? select length(11); select length(2.0); these sql will return errors, but hive is ok. this PR will support casting input types implicitly for function length the correct result is: select length(11) return 2 select length(2.0) return 3 Author: 岑玉海 <261810726@qq.com> Author: cenyuhai <cenyuhai@didichuxing.com> Closes #15014 from cenyuhai/SPARK-17429.
* [SPARK-17114][SQL] Fix aggregates grouped by literals with empty inputHerman van Hovell2016-09-152-0/+68
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes an issue with aggregates that have an empty input, and use a literals as their grouping keys. These aggregates are currently interpreted as aggregates **without** grouping keys, this triggers the ungrouped code path (which aways returns a single row). This PR fixes the `RemoveLiteralFromGroupExpressions` optimizer rule, which changes the semantics of the Aggregate by eliminating all literal grouping keys. ## How was this patch tested? Added tests to `SQLQueryTestSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15101 from hvanhovell/SPARK-17114-3.
* [SPARK-17440][SPARK-17441] Fixed Multiple Bugs in ALTER TABLEgatorsmile2016-09-151-14/+49
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? For the following `ALTER TABLE` DDL, we should issue an exception when the target table is a `VIEW`: ```SQL ALTER TABLE viewName SET LOCATION '/path/to/your/lovely/heart' ALTER TABLE viewName SET SERDE 'whatever' ALTER TABLE viewName SET SERDEPROPERTIES ('x' = 'y') ALTER TABLE viewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y') ALTER TABLE viewName ADD IF NOT EXISTS PARTITION (a='4', b='8') ALTER TABLE viewName DROP IF EXISTS PARTITION (a='2') ALTER TABLE viewName RECOVER PARTITIONS ALTER TABLE viewName PARTITION (a='1', b='q') RENAME TO PARTITION (a='100', b='p') ``` In addition, `ALTER TABLE RENAME PARTITION` is unable to handle data source tables, just like the other `ALTER PARTITION` commands. We should issue an exception instead. ### How was this patch tested? Added a few test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #15004 from gatorsmile/altertable.
* [SPARK-10747][SQL] Support NULLS FIRST|LAST clause in ORDER BYXin Wu2016-09-143-1/+339
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values. However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC). This PR is to support this new feature. ## How was this patch tested? New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Xin Wu <xinwu@us.ibm.com> Closes #14842 from xwu0226/SPARK-10747.
* [SPARK-17409][SQL] Do Not Optimize Query in CTAS More Than Oncegatorsmile2016-09-141-0/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? As explained in https://github.com/apache/spark/pull/14797: >Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs. For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result. We should not optimize the query in CTAS more than once. For example, ```Scala spark.range(99, 101).createOrReplaceTempView("tab1") val sqlStmt = "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1" sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt") checkAnswer(spark.table("tab2"), sql(sqlStmt)) ``` Before this PR, the results do not match ``` == Results == !== Correct Answer - 2 == == Spark Answer - 2 == ![100,100.000000000000000000] [100,null] [99,99.000000000000000000] [99,99.000000000000000000] ``` After this PR, the results match. ``` +---+----------------------+ |id |num | +---+----------------------+ |99 |99.000000000000000000 | |100|100.000000000000000000| +---+----------------------+ ``` In this PR, we do not treat the `query` in CTAS as a child. Thus, the `query` will not be optimized when optimizing CTAS statement. However, we still need to analyze it for normalizing and verifying the CTAS in the Analyzer. Thus, we do it in the analyzer rule `PreprocessDDL`, because so far only this rule needs the analyzed plan of the `query`. ### How was this patch tested? Added a test Author: gatorsmile <gatorsmile@gmail.com> Closes #15048 from gatorsmile/ctasOptimized.
* [SPARK-17445][DOCS] Reference an ASF page as the main place to find ↵Sean Owen2016-09-142-9/+6
| | | | | | | | | | | | | | | | | | third-party packages ## What changes were proposed in this pull request? Point references to spark-packages.org to https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects This will be accompanied by a parallel change to the spark-website repo, and additional changes to this wiki. ## How was this patch tested? Jenkins tests. Author: Sean Owen <sowen@cloudera.com> Closes #15075 from srowen/SPARK-17445.