aboutsummaryrefslogtreecommitdiff
path: root/sql/core
Commit message (Collapse)AuthorAgeFilesLines
* [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.
* [SPARK-16313][SQL] Spark should not silently drop exceptions in file listingReynold Xin2016-06-304-11/+27
| | | | | | | | | | | | ## What changes were proposed in this pull request? Spark silently drops exceptions during file listing. This is a very bad behavior because it can mask legitimate errors and the resulting plan will silently have 0 rows. This patch changes it to not silently drop the errors. ## How was this patch tested? Manually verified. Author: Reynold Xin <rxin@databricks.com> Closes #13987 from rxin/SPARK-16313.
* [SPARK-16336][SQL] Suggest doing table refresh upon FileNotFoundExceptionpetermaxlee2016-06-302-1/+102
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch appends a message to suggest users running refresh table or reloading data frames when Spark sees a FileNotFoundException due to stale, cached metadata. ## How was this patch tested? Added a unit test for this in MetadataCacheSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14003 from petermaxlee/SPARK-16336.
* [SPARK-16289][SQL] Implement posexplode table generating functionDongjoon Hyun2016-06-304-60/+101
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive. **Before** ```scala scala> sql("select posexplode(map('a', 1, 'b', 2))").show org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7 ``` **After** ```scala scala> sql("select posexplode(map('a', 1, 'b', 2))").show +---+---+-----+ |pos|key|value| +---+---+-----+ | 0| a| 1| | 1| b| 2| +---+---+-----+ ``` For `array` argument, `after` is the same with `before`. ``` scala> sql("select posexplode(array(1, 2, 3))").show +---+---+ |pos|col| +---+---+ | 0| 1| | 1| 2| | 2| 3| +---+---+ ``` ## How was this patch tested? Pass the Jenkins tests with newly added testcases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13971 from dongjoon-hyun/SPARK-16289.
* [SPARK-13850] Force the sorter to Spill when number of elements in th…Sital Kedia2016-06-307-5/+24
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Force the sorter to Spill when number of elements in the pointer array reach a certain size. This is to workaround the issue of timSort failing on large buffer size. ## How was this patch tested? Tested by running a job which was failing without this change due to TimSort bug. Author: Sital Kedia <skedia@fb.com> Closes #13107 from sitalkedia/fix_TimSort.
* [SPARK-15820][PYSPARK][SQL] Add Catalog.refreshTable into python APIWeichenXu2016-06-301-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add Catalog.refreshTable API into python interface for Spark-SQL. ## How was this patch tested? Existing test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #13558 from WeichenXu123/update_python_sql_interface_refreshTable.
* [SPARK-16274][SQL] Implement xpath_booleanpetermaxlee2016-06-301-0/+32
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch implements xpath_boolean expression for Spark SQL, a xpath function that returns true or false. The implementation is modelled after Hive's xpath_boolean, except that how the expression handles null inputs. Hive throws a NullPointerException at runtime if either of the input is null. This implementation returns null if either of the input is null. ## How was this patch tested? Created two new test suites. One for unit tests covering the expression, and the other for end-to-end test in SQL. Author: petermaxlee <petermaxlee@gmail.com> Closes #13964 from petermaxlee/SPARK-16274.
* [SPARK-16267][TEST] Replace deprecated `CREATE TEMPORARY TABLE ... USING` ↵Dongjoon Hyun2016-06-295-11/+11
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | from testsuites. ## What changes were proposed in this pull request? After SPARK-15674, `DDLStrategy` prints out the following deprecation messages in the testsuites. ``` 12:10:53.284 WARN org.apache.spark.sql.execution.SparkStrategies$DDLStrategy: CREATE TEMPORARY TABLE normal_orc_source USING... is deprecated, please use CREATE TEMPORARY VIEW viewName USING... instead ``` Total : 40 - JDBCWriteSuite: 14 - DDLSuite: 6 - TableScanSuite: 6 - ParquetSourceSuite: 5 - OrcSourceSuite: 2 - SQLQuerySuite: 2 - HiveCommandSuite: 2 - JsonSuite: 1 - PrunedScanSuite: 1 - FilteredScanSuite 1 This PR replaces `CREATE TEMPORARY TABLE` with `CREATE TEMPORARY VIEW` in order to remove the deprecation messages in the above testsuites except `DDLSuite`, `SQLQuerySuite`, `HiveCommandSuite`. The Jenkins results shows only remaining 10 messages. https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61422/consoleFull ## How was this patch tested? This is a testsuite-only change. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13956 from dongjoon-hyun/SPARK-16267.
* [SPARK-16134][SQL] optimizer rules for typed filterWenchen Fan2016-06-303-10/+5
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds 3 optimizer rules for typed filter: 1. push typed filter down through `SerializeFromObject` and eliminate the deserialization in filter condition. 2. pull typed filter up through `SerializeFromObject` and eliminate the deserialization in filter condition. 3. combine adjacent typed filters and share the deserialized object among all the condition expressions. This PR also adds `TypedFilter` logical plan, to separate it from normal filter, so that the concept is more clear and it's easier to write optimizer rules. ## How was this patch tested? `TypedFilterOptimizationSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #13846 from cloud-fan/filter.
* [SPARK-16006][SQL] Attemping to write empty DataFrame with no fields throws ↵Dongjoon Hyun2016-06-292-2/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | non-intuitive exception ## What changes were proposed in this pull request? This PR allows `emptyDataFrame.write` since the user didn't specify any partition columns. **Before** ```scala scala> spark.emptyDataFrame.write.parquet("/tmp/t1") org.apache.spark.sql.AnalysisException: Cannot use all columns for partition columns; scala> spark.emptyDataFrame.write.csv("/tmp/t1") org.apache.spark.sql.AnalysisException: Cannot use all columns for partition columns; ``` After this PR, there occurs no exceptions and the created directory has only one file, `_SUCCESS`, as expected. ## How was this patch tested? Pass the Jenkins tests including updated test cases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13730 from dongjoon-hyun/SPARK-16006.
* [SPARK-14480][SQL] Remove meaningless StringIteratorReader for CSV data source.hyukjinkwon2016-06-294-291/+33
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR removes meaningless `StringIteratorReader` for CSV data source. In `CSVParser.scala`, there is an `Reader` wrapping `Iterator` but there are two problems by this. Firstly, it was actually not faster than processing line by line with Iterator due to additional logics to wrap `Iterator` to `Reader`. Secondly, this brought a bit of complexity because it needs additional logics to allow every line to be read bytes by bytes. So, it was pretty difficult to figure out issues about parsing, (eg. SPARK-14103). A benchmark was performed manually and the results were below: - Original codes with Reader wrapping Iterator |End-to-end (ns) | Parse Time (ns) | |-----------------------|------------------------| |14116265034 |2008277960 | - New codes with Iterator |End-to-end (ns) | Parse Time (ns) | |-----------------------|------------------------| |13451699644 | 1549050564 | For the details for the environment, dataset and methods, please refer the JIRA ticket. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #13808 from HyukjinKwon/SPARK-14480-small.
* [SPARK-16157][SQL] Add New Methods for comments in StructField and StructTypegatorsmile2016-06-295-13/+31
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? Based on the previous discussion with cloud-fan hvanhovell in another related PR https://github.com/apache/spark/pull/13764#discussion_r67994276, it looks reasonable to add convenience methods for users to add `comment` when defining `StructField`. Currently, the column-related `comment` attribute is stored in `Metadata` of `StructField`. For example, users can add the `comment` attribute using the following way: ```Scala StructType( StructField( "cl1", IntegerType, nullable = false, new MetadataBuilder().putString("comment", "test").build()) :: Nil) ``` This PR is to add more user friendly methods for the `comment` attribute when defining a `StructField`. After the changes, users are provided three different ways to do it: ```Scala val struct = (new StructType) .add("a", "int", true, "test1") val struct = (new StructType) .add("c", StringType, true, "test3") val struct = (new StructType) .add(StructField("d", StringType).withComment("test4")) ``` #### How was this patch tested? Added test cases: - `DataTypeSuite` is for testing three types of API changes, - `DataFrameReaderWriterSuite` is for parquet, json and csv formats - using in-memory catalog - `OrcQuerySuite.scala` is for orc format using Hive-metastore Author: gatorsmile <gatorsmile@gmail.com> Closes #13860 from gatorsmile/newMethodForComment.
* [SPARK-16291][SQL] CheckAnalysis should capture nested aggregate functions ↵Cheng Lian2016-06-291-3/+1
| | | | | | | | | | | | | | | | | | | | that reference no input attributes ## What changes were proposed in this pull request? `MAX(COUNT(*))` is invalid since aggregate expression can't be nested within another aggregate expression. This case should be captured at analysis phase, but somehow sneaks off to runtime. The reason is that when checking aggregate expressions in `CheckAnalysis`, a checking branch treats all expressions that reference no input attributes as valid ones. However, `MAX(COUNT(*))` is translated into `MAX(COUNT(1))` at analysis phase and also references no input attribute. This PR fixes this issue by removing the aforementioned branch. ## How was this patch tested? New test case added in `AnalysisErrorSuite`. Author: Cheng Lian <lian@databricks.com> Closes #13968 from liancheng/spark-16291-nested-agg-functions.
* [TRIVIAL][DOCS][STREAMING][SQL] The return type mentioned in the Javadoc is ↵Holden Karau2016-06-291-2/+2
| | | | | | | | | | | | | | | | incorrect for toJavaRDD, … ## What changes were proposed in this pull request? Change the return type mentioned in the JavaDoc for `toJavaRDD` / `javaRDD` to match the actual return type & be consistent with the scala rdd return type. ## How was this patch tested? Docs only change. Author: Holden Karau <holden@us.ibm.com> Closes #13954 from holdenk/trivial-streaming-tojavardd-doc-fix.
* [MINOR][DOCS][STRUCTURED STREAMING] Minor doc fixes around `DataFrameWriter` ↵Burak Yavuz2016-06-285-9/+9
| | | | | | | | | | | | and `DataStreamWriter` ## What changes were proposed in this pull request? Fixes a couple old references to `DataFrameWriter.startStream` to `DataStreamWriter.start Author: Burak Yavuz <brkyvz@gmail.com> Closes #13952 from brkyvz/minor-doc-fix.
* [SPARK-16100][SQL] fix bug when use Map as the buffer type of AggregatorWenchen Fan2016-06-291-0/+15
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The root cause is in `MapObjects`. Its parameter `loopVar` is not declared as child, but sometimes can be same with `lambdaFunction`(e.g. the function that takes `loopVar` and produces `lambdaFunction` may be `identity`), which is a child. This brings trouble when call `withNewChildren`, it may mistakenly treat `loopVar` as a child and cause `IndexOutOfBoundsException: 0` later. This PR fixes this bug by simply pulling out the paremters from `LambdaVariable` and pass them to `MapObjects` directly. ## How was this patch tested? new test in `DatasetAggregatorSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #13835 from cloud-fan/map-objects.
* [SPARK-16236][SQL] Add Path Option back to Load API in DataFrameReadergatorsmile2016-06-282-1/+29
| | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? koertkuipers identified the PR https://github.com/apache/spark/pull/13727/ changed the behavior of `load` API. After the change, the `load` API does not add the value of `path` into the `options`. Thank you! This PR is to add the option `path` back to `load()` API in `DataFrameReader`, if and only if users specify one and only one `path` in the `load` API. For example, users can see the `path` option after the following API call, ```Scala spark.read .format("parquet") .load("/test") ``` #### How was this patch tested? Added test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #13933 from gatorsmile/optionPath.
* [SPARK-16181][SQL] outer join with isNull filter may return wrong resultWenchen Fan2016-06-281-0/+9
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The root cause is: the output attributes of outer join are derived from its children, while they are actually different attributes(outer join can return null). We have already added some special logic to handle it, e.g. `PushPredicateThroughJoin` won't push down predicates through outer join side, `FixNullability`. This PR adds one more special logic in `FoldablePropagation`. ## How was this patch tested? new test in `DataFrameSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #13884 from cloud-fan/bug.