aboutsummaryrefslogtreecommitdiff
path: root/sql/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-16482][SQL] Describe Table Command for Tables Requiring Runtime ↵gatorsmile2016-07-131-15/+13
| | | | | | | | | | | | | | | | | | 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-16531][SQL][TEST] Remove timezone setting from ↵Burak Yavuz2016-07-131-10/+0
| | | | | | | | | | | | DataFrameTimeWindowingSuite ## What changes were proposed in this pull request? It's unnecessary. `QueryTest` already sets it. Author: Burak Yavuz <brkyvz@gmail.com> Closes #14170 from brkyvz/test-tz.
* [SPARK-16439] Fix number formatting in SQL UIMaciej Brynski2016-07-131-1/+3
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Spark SQL UI display numbers greater than 1000 with u00A0 as grouping separator. Problem exists when server locale has no-breaking space as separator. (for example pl_PL) This patch turns off grouping and remove this separator. The problem starts with this PR. https://github.com/apache/spark/pull/12425/files#diff-803f475b01acfae1c5c96807c2ea9ddcR125 ## How was this patch tested? Manual UI tests. Screenshot attached. ![image](https://cloud.githubusercontent.com/assets/4006010/16749556/5cb5a372-47cb-11e6-9a95-67fd3f9d1c71.png) Author: Maciej Brynski <maciej.brynski@adpilot.pl> Closes #14142 from maver1ck/master.
* [MINOR] Fix Java style errors and remove unused importsXin Ren2016-07-132-3/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix Java style errors and remove unused imports, which are randomly found ## How was this patch tested? Tested on my local machine. Author: Xin Ren <iamshrek@126.com> Closes #14161 from keypointt/SPARK-16437.
* [SPARK-15889][STREAMING] Follow-up fix to erroneous condition in StreamTestSean Owen2016-07-131-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? A second form of AssertQuery now actually invokes the condition; avoids a build warning too ## How was this patch tested? Jenkins; running StreamTest Author: Sean Owen <sowen@cloudera.com> Closes #14133 from srowen/SPARK-15889.2.
* [SPARK-16284][SQL] Implement reflect SQL functionpetermaxlee2016-07-131-0/+38
| | | | | | | | | | | | | | ## 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-128-45/+54
| | | | | | | | | | | | | | | | | | | | | | | | | 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-16489][SQL] Guard against variable reuse mistakes in expression code ↵Reynold Xin2016-07-121-14/+0
| | | | | | | | | | | | | | | | generation ## What changes were proposed in this pull request? In code generation, it is incorrect for expressions to reuse variable names across different instances of itself. As an example, SPARK-16488 reports a bug in which pmod expression reuses variable name "r". This patch updates ExpressionEvalHelper test harness to always project two instances of the same expression, which will help us catch variable reuse problems in expression unit tests. This patch also fixes the bug in crc32 expression. ## How was this patch tested? This is a test harness change, but I also created a new test suite for testing the test harness. Author: Reynold Xin <rxin@databricks.com> Closes #14146 from rxin/SPARK-16489.
* [SPARK-15752][SQL] Optimize metadata only query that has an aggregate whose ↵Lianhui Wang2016-07-125-0/+319
| | | | | | | | | | | | | | | | 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-16189][SQL] Add ExternalRDD logical plan for input with RDD to have a ↵Takuya UESHIN2016-07-125-20/+81
| | | | | | | | | | | | | | | | | | | chance to eliminate serialize/deserialize. ## What changes were proposed in this pull request? Currently the input `RDD` of `Dataset` is always serialized to `RDD[InternalRow]` prior to being as `Dataset`, but there is a case that we use `map` or `mapPartitions` just after converted to `Dataset`. In this case, serialize and then deserialize happens but it would not be needed. This pr adds `ExistingRDD` logical plan for input with `RDD` to have a chance to eliminate serialize/deserialize. ## How was this patch tested? Existing tests. Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #13890 from ueshin/issues/SPARK-16189.
* [SPARK-16199][SQL] Add a method to list the referenced columns in data ↵petermaxlee2016-07-112-15/+145
| | | | | | | | | | | | | | | | | source Filter ## What changes were proposed in this pull request? It would be useful to support listing the columns that are referenced by a filter. This can help simplify data source planning, because with this we would be able to implement unhandledFilters method in HadoopFsRelation. This is based on rxin's patch (#13901) and adds unit tests. ## How was this patch tested? Added a new suite FiltersSuite. Author: petermaxlee <petermaxlee@gmail.com> Author: Reynold Xin <rxin@databricks.com> Closes #14120 from petermaxlee/SPARK-16199.
* [SPARK-12639][SQL] Mark Filters Fully Handled By Sources with *Russell Spitzer2016-07-112-9/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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-16488] Fix codegen variable namespace collision in pmod and partitionBySameer Agarwal2016-07-111-0/+14
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes a variable namespace collision bug in pmod and partitionBy ## How was this patch tested? Regression test for one possible occurrence. A more general fix in `ExpressionEvalHelper.checkEvaluation` will be in a subsequent PR. Author: Sameer Agarwal <sameer@databricks.com> Closes #14144 from sameeragarwal/codegen-bug.
* [SPARK-16430][SQL][STREAMING] Fixed bug in the maxFilesPerTrigger in ↵Tathagata Das2016-07-112-5/+36
| | | | | | | | | | | | | | | | FileStreamSource ## What changes were proposed in this pull request? Incorrect list of files were being allocated to a batch. This caused a file to read multiple times in the multiple batches. ## How was this patch tested? Added unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14143 from tdas/SPARK-16430-1.
* [SPARK-16433][SQL] Improve StreamingQuery.explain when no data arrivesShixiong Zhu2016-07-113-5/+5
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Display `No physical plan. Waiting for data.` instead of `N/A` for StreamingQuery.explain when no data arrives because `N/A` doesn't provide meaningful information. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #14100 from zsxwing/SPARK-16433.
* [SPARK-16114][SQL] structured streaming event time window exampleJames Thomas2016-07-112-10/+89
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? A structured streaming example with event time windowing. ## How was this patch tested? Run locally Author: James Thomas <jamesjoethomas@gmail.com> Closes #13957 from jjthomas/current.
* [SPARK-16458][SQL] SessionCatalog should support `listColumns` for temporary ↵Dongjoon Hyun2016-07-113-3/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | tables ## What changes were proposed in this pull request? Temporary tables are used frequently, but `spark.catalog.listColumns` does not support those tables. This PR make `SessionCatalog` supports temporary table column listing. **Before** ```scala scala> spark.range(10).createOrReplaceTempView("t1") scala> spark.catalog.listTables().collect() res1: Array[org.apache.spark.sql.catalog.Table] = Array(Table[name=`t1`, tableType=`TEMPORARY`, isTemporary=`true`]) scala> spark.catalog.listColumns("t1").collect() org.apache.spark.sql.AnalysisException: Table `t1` does not exist in database `default`.; ``` **After** ``` scala> spark.catalog.listColumns("t1").collect() res2: Array[org.apache.spark.sql.catalog.Column] = Array(Column[name='id', description='id', dataType='bigint', nullable='false', isPartition='false', isBucket='false']) ``` ## How was this patch tested? Pass the Jenkins tests including a new testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14114 from dongjoon-hyun/SPARK-16458.
* [SPARK-16477] Bump master version to 2.1.0-SNAPSHOTReynold Xin2016-07-111-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? After SPARK-16476 (committed earlier today as #14128), we can finally bump the version number. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #14130 from rxin/SPARK-16477.
* [SPARK-16459][SQL] Prevent dropping current databaseDongjoon Hyun2016-07-111-0/+9
| | | | | | | | | | | | | | | | | | | | | | ## 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-16355][SPARK-16354][SQL] Fix Bugs When LIMIT/TABLESAMPLE is ↵gatorsmile2016-07-112-2/+79
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Non-foldable, Zero or Negative #### What changes were proposed in this pull request? **Issue 1:** When a query containing LIMIT/TABLESAMPLE 0, the statistics could be zero. Results are correct but it could cause a huge performance regression. For example, ```Scala Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") .createOrReplaceTempView("test") val df1 = spark.table("test") val df2 = spark.table("test").limit(0) val df = df1.join(df2, Seq("k"), "left") ``` The statistics of both `df` and `df2` are zero. The statistics values should never be zero; otherwise `sizeInBytes` of `BinaryNode` will also be zero (product of children). This PR is to increase it to `1` when the num of rows is equal to 0. **Issue 2:** When a query containing negative LIMIT/TABLESAMPLE, we should issue exceptions. Negative values could break the implementation assumption of multiple parts. For example, statistics calculation. Below is the example query. ```SQL SELECT * FROM testData TABLESAMPLE (-1 rows) SELECT * FROM testData LIMIT -1 ``` This PR is to issue an appropriate exception in this case. **Issue 3:** Spark SQL follows the restriction of LIMIT clause in Hive. The argument to the LIMIT clause must evaluate to a constant value. It can be a numeric literal, or another kind of numeric expression involving operators, casts, and function return values. You cannot refer to a column or use a subquery. Currently, we do not detect whether the expression in LIMIT clause is foldable or not. If non-foldable, we might issue a strange error message. For example, ```SQL SELECT * FROM testData LIMIT rand() > 0.2 ``` Then, a misleading error message is issued, like ``` assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2) +- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203] +- LocalLimit (_nondeterministic#202 > 0.2) +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202] +- LogicalRDD [key#11, value#12] java.lang.AssertionError: assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2) +- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203] +- LocalLimit (_nondeterministic#202 > 0.2) +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202] +- LogicalRDD [key#11, value#12] ``` This PR detects it and then issues a meaningful error message. #### How was this patch tested? Added test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #14034 from gatorsmile/limit.
* [SPARK-16318][SQL] Implement all remaining xpath functionspetermaxlee2016-07-112-32/+62
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch implements all remaining xpath functions that Hive supports and not natively supported in Spark: xpath_int, xpath_short, xpath_long, xpath_float, xpath_double, xpath_string, and xpath. ## How was this patch tested? Added unit tests and end-to-end tests. Author: petermaxlee <petermaxlee@gmail.com> Closes #13991 from petermaxlee/SPARK-16318.
* [SPARK-16401][SQL] Data Source API: Enable Extending RelationProvider and ↵gatorsmile2016-07-092-3/+34
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | CreatableRelationProvider without Extending SchemaRelationProvider #### What changes were proposed in this pull request? When users try to implement a data source API with extending only `RelationProvider` and `CreatableRelationProvider`, they will hit an error when resolving the relation. ```Scala spark.read .format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema") .load() .write. format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema") .save() ``` The error they hit is like ``` org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema does not allow user-specified schemas.; org.apache.spark.sql.AnalysisException: org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema does not allow user-specified schemas.; at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:319) at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:494) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:211) ``` Actually, the bug fix is simple. [`DataSource.createRelation(sparkSession.sqlContext, mode, options, data)`](https://github.com/gatorsmile/spark/blob/dd644f8117e889cebd6caca58702a7c7e3d88bef/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L429) already returns a BaseRelation. We should not assign schema to `userSpecifiedSchema`. That schema assignment only makes sense for the data sources that extend `FileFormat`. #### How was this patch tested? Added a test case. Author: gatorsmile <gatorsmile@gmail.com> Closes #14075 from gatorsmile/dataSource.
* [SPARK-16387][SQL] JDBC Writer should use dialect to quote field names.Dongjoon Hyun2016-07-082-4/+11
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, JDBC Writer uses dialects to get datatypes, but doesn't to quote field names. This PR uses dialects to quote the field names, too. **Reported Error Scenario (MySQL case)** ```scala scala> val url="jdbc:mysql://localhost:3306/temp" scala> val prop = new java.util.Properties scala> prop.setProperty("user","root") scala> spark.createDataset(Seq("a","b","c")).toDF("order") scala> df.write.mode("overwrite").jdbc(url, "temptable", prop) ...MySQLSyntaxErrorException: ... near 'order TEXT ) ``` ## How was this patch tested? Pass the Jenkins tests and manually do the above case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14107 from dongjoon-hyun/SPARK-16387.
* [SPARK-16281][SQL] Implement parse_url SQL functionwujian2016-07-081-0/+15
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds parse_url SQL functions in order to remove Hive fallback. A new implementation of #13999 ## How was this patch tested? Pass the exist tests including new testcases. Author: wujian <jan.chou.wu@gmail.com> Closes #14008 from janplus/SPARK-16281.
* [SPARK-16429][SQL] Include `StringType` columns in `describe()`Dongjoon Hyun2016-07-082-21/+31
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, Spark `describe` supports `StringType`. However, `describe()` returns a dataset for only all numeric columns. This PR aims to include `StringType` columns in `describe()`, `describe` without argument. **Background** ```scala scala> spark.read.json("examples/src/main/resources/people.json").describe("age", "name").show() +-------+------------------+-------+ |summary| age| name| +-------+------------------+-------+ | count| 2| 3| | mean| 24.5| null| | stddev|7.7781745930520225| null| | min| 19| Andy| | max| 30|Michael| +-------+------------------+-------+ ``` **Before** ```scala scala> spark.read.json("examples/src/main/resources/people.json").describe().show() +-------+------------------+ |summary| age| +-------+------------------+ | count| 2| | mean| 24.5| | stddev|7.7781745930520225| | min| 19| | max| 30| +-------+------------------+ ``` **After** ```scala scala> spark.read.json("examples/src/main/resources/people.json").describe().show() +-------+------------------+-------+ |summary| age| name| +-------+------------------+-------+ | count| 2| 3| | mean| 24.5| null| | stddev|7.7781745930520225| null| | min| 19| Andy| | max| 30|Michael| +-------+------------------+-------+ ``` ## How was this patch tested? Pass the Jenkins with a update testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14095 from dongjoon-hyun/SPARK-16429.
* [SPARK-13638][SQL] Add quoteAll option to CSV DataFrameWriterJurriaan Pruis2016-07-084-1/+31
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Adds an quoteAll option for writing CSV which will quote all fields. See https://issues.apache.org/jira/browse/SPARK-13638 ## How was this patch tested? Added a test to verify the output columns are quoted for all fields in the Dataframe Author: Jurriaan Pruis <email@jurriaanpruis.nl> Closes #13374 from jurriaan/csv-quote-all.
* [SPARK-16285][SQL] Implement sentences SQL functionsDongjoon Hyun2016-07-081-0/+20
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `sentences` SQL function. ## How was this patch tested? Pass the Jenkins tests with a new testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14004 from dongjoon-hyun/SPARK_16285.
* [SPARK-16430][SQL][STREAMING] Add option maxFilesPerTriggerTathagata Das2016-07-073-14/+112
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? An option that limits the file stream source to read 1 file at a time enables rate limiting. It has the additional convenience that a static set of files can be used like a stream for testing as this will allows those files to be considered one at a time. This PR adds option `maxFilesPerTrigger`. ## How was this patch tested? New unit test Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14094 from tdas/SPARK-16430.
* [SPARK-16350][SQL] Fix support for incremental planning in wirteStream.foreach()Liwei Lin2016-07-073-13/+117
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? There are cases where `complete` output mode does not output updated aggregated value; for details please refer to [SPARK-16350](https://issues.apache.org/jira/browse/SPARK-16350). The cause is that, as we do `data.as[T].foreachPartition { iter => ... }` in `ForeachSink.addBatch()`, `foreachPartition()` does not support incremental planning for now. This patches makes `foreachPartition()` support incremental planning in `ForeachSink`, by making a special version of `Dataset` with its `rdd()` method supporting incremental planning. ## How was this patch tested? Added a unit test which failed before the change Author: Liwei Lin <lwlin7@gmail.com> Closes #14030 from lw-lin/fix-foreach-complete.
* [SPARK-16400][SQL] Remove InSet filter pushdown from ParquetReynold Xin2016-07-072-75/+12
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch removes InSet filter pushdown from Parquet data source, since row-based pushdown is not beneficial to Spark and brings extra complexity to the code base. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #14076 from rxin/SPARK-16400.
* [SPARK-16368][SQL] Fix Strange Errors When Creating View With Unmatched ↵gatorsmile2016-07-072-1/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-14839][SQL] Support for other types for `tableProperty` rule in SQL ↵hyukjinkwon2016-07-062-4/+79
| | | | | | | | | | | | | | | | | | | | syntax ## What changes were proposed in this pull request? Currently, Scala API supports to take options with the types, `String`, `Long`, `Double` and `Boolean` and Python API also supports other types. This PR corrects `tableProperty` rule to support other types (string, boolean, double and integer) so that support the options for data sources in a consistent way. This will affect other rules such as DBPROPERTIES and TBLPROPERTIES (allowing other types as values). Also, `TODO add bucketing and partitioning.` was removed because it was resolved in https://github.com/apache/spark/commit/24bea000476cdd0b43be5160a76bc5b170ef0b42 ## How was this patch tested? Unit test in `MetastoreDataSourcesSuite.scala`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #13517 from HyukjinKwon/SPARK-14839.
* [SPARK-16371][SQL] Two follow-up tasksReynold Xin2016-07-061-1/+1
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is a small follow-up for SPARK-16371: 1. Hide removeMetadata from public API. 2. Add JIRA ticket number to test case name. ## How was this patch tested? Updated a test comment. Author: Reynold Xin <rxin@databricks.com> Closes #14074 from rxin/parquet-filter.
* [SPARK-16371][SQL] Do not push down filters incorrectly when inner name and ↵hyukjinkwon2016-07-063-2/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | outer name are the same in Parquet ## What changes were proposed in this pull request? Currently, if there is a schema as below: ``` root |-- _1: struct (nullable = true) | |-- _1: integer (nullable = true) ``` and if we execute the codes below: ```scala df.filter("_1 IS NOT NULL").count() ``` This pushes down a filter although this filter is being applied to `StructType`.(If my understanding is correct, Spark does not pushes down filters for those). The reason is, `ParquetFilters.getFieldMap` produces results below: ``` (_1,StructType(StructField(_1,IntegerType,true))) (_1,IntegerType) ``` and then it becomes a `Map` ``` (_1,IntegerType) ``` Now, because of ` ....lift(dataTypeOf(name)).map(_(name, value))`, this pushes down filters for `_1` which Parquet thinks is `IntegerType`. However, it is actually `StructType`. So, Parquet filter2 produces incorrect results, for example, the codes below: ``` df.filter("_1 IS NOT NULL").count() ``` produces always 0. This PR prevents this by not finding nested fields. ## How was this patch tested? Unit test in `ParquetFilterSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14067 from HyukjinKwon/SPARK-16371.
* [SPARK-15979][SQL] Renames CatalystWriteSupport to ParquetWriteSupportCheng Lian2016-07-062-10/+10
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? PR #13696 renamed various Parquet support classes but left `CatalystWriteSupport` behind. This PR is renames it as a follow-up. ## How was this patch tested? N/A. Author: Cheng Lian <lian@databricks.com> Closes #14070 from liancheng/spark-15979-follow-up.
* [SPARK-16388][SQL] Remove spark.sql.nativeView and ↵Reynold Xin2016-07-063-60/+18
| | | | | | | | | | | | | | 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-16340][SQL] Support column arguments for `regexp_replace` Dataset ↵Dongjoon Hyun2016-07-052-1/+13
| | | | | | | | | | | | | | | | operation ## What changes were proposed in this pull request? Currently, `regexp_replace` function supports `Column` arguments in a query. This PR supports that in a `Dataset` operation, too. ## How was this patch tested? Pass the Jenkins tests with a updated testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14060 from dongjoon-hyun/SPARK-16340.
* [SPARK-16286][SQL] Implement stack table generating functionDongjoon Hyun2016-07-061-0/+53
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `stack` table generating function. ## How was this patch tested? Pass the Jenkins tests including new testcases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14033 from dongjoon-hyun/SPARK-16286.
* [SPARK-16383][SQL] Remove `SessionState.executeSql`Dongjoon Hyun2016-07-051-3/+1
| | | | | | | | | | | | | | ## 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-055-9/+14
| | | | | | | | | | | | | | | ## 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-9876][SQL][FOLLOWUP] Enable string and binary tests for Parquet ↵hyukjinkwon2016-07-054-18/+9
| | | | | | | | | | | | | | | | | | predicate pushdown and replace deprecated fromByteArray. ## What changes were proposed in this pull request? It seems Parquet has been upgraded to 1.8.1 by https://github.com/apache/spark/pull/13280. So, this PR enables string and binary predicate push down which was disabled due to [SPARK-11153](https://issues.apache.org/jira/browse/SPARK-11153) and [PARQUET-251](https://issues.apache.org/jira/browse/PARQUET-251) and cleans up some comments unremoved (I think by mistake). This PR also replace the API, `fromByteArray()` deprecated in [PARQUET-251](https://issues.apache.org/jira/browse/PARQUET-251). ## How was this patch tested? Unit tests in `ParquetFilters` Author: hyukjinkwon <gurwls223@gmail.com> Closes #13389 from HyukjinKwon/parquet-1.8-followup.
* [SPARK-16360][SQL] Speed up SQL query performance by removing redundant ↵Dongjoon Hyun2016-07-051-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | `executePlan` call ## What changes were proposed in this pull request? Currently, there are a few reports about Spark 2.0 query performance regression for large queries. This PR speeds up SQL query processing performance by removing redundant **consecutive `executePlan`** call in `Dataset.ofRows` function and `Dataset` instantiation. Specifically, this PR aims to reduce the overhead of SQL query execution plan generation, not real query execution. So, we can not see the result in the Spark Web UI. Please use the following query script. The result is **25.78 sec** -> **12.36 sec** as expected. **Sample Query** ```scala val n = 4000 val values = (1 to n).map(_.toString).mkString(", ") val columns = (1 to n).map("column" + _).mkString(", ") val query = s""" |SELECT $columns |FROM VALUES ($values) T($columns) |WHERE 1=2 AND 1 IN ($columns) |GROUP BY $columns |ORDER BY $columns |""".stripMargin def time[R](block: => R): R = { val t0 = System.nanoTime() val result = block println("Elapsed time: " + ((System.nanoTime - t0) / 1e9) + "s") result } ``` **Before** ```scala scala> time(sql(query)) Elapsed time: 30.138142577s // First query has a little overhead of initialization. res0: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields] scala> time(sql(query)) Elapsed time: 25.787751452s // Let's compare this one. res1: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields] ``` **After** ```scala scala> time(sql(query)) Elapsed time: 17.500279659s // First query has a little overhead of initialization. res0: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields] scala> time(sql(query)) Elapsed time: 12.364812255s // This shows the real difference. The speed up is about 2 times. res1: org.apache.spark.sql.DataFrame = [column1: int, column2: int ... 3998 more fields] ``` ## How was this patch tested? Manual by the above script. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14044 from dongjoon-hyun/SPARK-16360.
* [SPARK-15204][SQL] improve nullability inference for AggregatorKoert Kuipers2016-07-042-4/+13
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? TypedAggregateExpression sets nullable based on the schema of the outputEncoder ## How was this patch tested? Add test in DatasetAggregatorSuite Author: Koert Kuipers <koert@tresata.com> Closes #13532 from koertkuipers/feat-aggregator-nullable.
* [SPARK-16288][SQL] Implement inline table generating functionDongjoon Hyun2016-07-041-0/+60
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `inline` table generating function. ## How was this patch tested? Pass the Jenkins tests with new testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13976 from dongjoon-hyun/SPARK-16288.
* [SPARK-16278][SPARK-16279][SQL] Implement map_keys/map_values SQL functionsDongjoon Hyun2016-07-031-0/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds `map_keys` and `map_values` SQL functions in order to remove Hive fallback. ## How was this patch tested? Pass the Jenkins tests including new testcases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13967 from dongjoon-hyun/SPARK-16278.
* [SPARK-16329][SQL] Star Expansion over Table Containing No Columngatorsmile2016-07-031-0/+31
| | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? Star expansion over a table containing zero column does not work since 1.6. However, it works in Spark 1.5.1. This PR is to fix the issue in the master branch. For example, ```scala val rddNoCols = sqlContext.sparkContext.parallelize(1 to 10).map(_ => Row.empty) val dfNoCols = sqlContext.createDataFrame(rddNoCols, StructType(Seq.empty)) dfNoCols.registerTempTable("temp_table_no_cols") sqlContext.sql("select * from temp_table_no_cols").show ``` Without the fix, users will get the following the exception: ``` java.lang.IllegalArgumentException: requirement failed at scala.Predef$.require(Predef.scala:221) at org.apache.spark.sql.catalyst.analysis.UnresolvedStar.expand(unresolved.scala:199) ``` #### How was this patch tested? Tests are added Author: gatorsmile <gatorsmile@gmail.com> Closes #14007 from gatorsmile/starExpansionTableWithZeroColumn.
* [MINOR][BUILD] Fix Java linter errorsDongjoon Hyun2016-07-022-3/+4
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes the minor Java linter errors like the following. ``` - public int read(char cbuf[], int off, int len) throws IOException { + public int read(char[] cbuf, int off, int len) throws IOException { ``` ## How was this patch tested? Manual. ``` $ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install $ dev/lint-java Using `mvn` from path: /usr/local/bin/mvn Checkstyle checks passed. ``` Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14017 from dongjoon-hyun/minor_build_java_linter_error.
* [SPARK-16335][SQL] Structured streaming should fail if source directory does ↵Reynold Xin2016-07-012-23/+27
| | | | | | | | | | | | | | not exist ## What changes were proposed in this pull request? In structured streaming, Spark does not report errors when the specified directory does not exist. This is a behavior different from the batch mode. This patch changes the behavior to fail if the directory does not exist (when the path is not a glob pattern). ## How was this patch tested? Updated unit tests to reflect the new behavior. Author: Reynold Xin <rxin@databricks.com> Closes #14002 from rxin/SPARK-16335.
* [SPARK-16222][SQL] JDBC Sources - Handling illegal input values for ↵gatorsmile2016-07-017-45/+98
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | `fetchsize` and `batchsize` #### What changes were proposed in this pull request? For JDBC data sources, users can specify `batchsize` for multi-row inserts and `fetchsize` for multi-row fetch. A few issues exist: - The property keys are case sensitive. Thus, the existing test cases for `fetchsize` use incorrect names, `fetchSize`. Basically, the test cases are broken. - No test case exists for `batchsize`. - We do not detect the illegal input values for `fetchsize` and `batchsize`. For example, when `batchsize` is zero, we got the following exception: ``` Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.ArithmeticException: / by zero ``` when `fetchsize` is less than zero, we got the exception from the underlying JDBC driver: ``` Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): org.h2.jdbc.JdbcSQLException: Invalid value "-1" for parameter "rows" [90008-183] ``` This PR fixes all the above issues, and issue the appropriate exceptions when detecting the illegal inputs for `fetchsize` and `batchsize`. Also update the function descriptions. #### How was this patch tested? Test cases are fixed and added. Author: gatorsmile <gatorsmile@gmail.com> Closes #13919 from gatorsmile/jdbcProperties.
* [SPARK-16276][SQL] Implement elt SQL functionpetermaxlee2016-07-011-0/+14
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch implements the elt function, as it is implemented in Hive. ## How was this patch tested? Added expression unit test in StringExpressionsSuite and end-to-end test in StringFunctionsSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #13966 from petermaxlee/SPARK-16276.