aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-16686][SQL] Remove PushProjectThroughSample since it is handled by ↵Liang-Chi Hsieh2016-07-261-0/+25
| | | | | | | | | | | | | | | | | | ColumnPruning ## What changes were proposed in this pull request? We push down `Project` through `Sample` in `Optimizer` by the rule `PushProjectThroughSample`. However, if the projected columns produce new output, they will encounter whole data instead of sampled data. It will bring some inconsistency between original plan (Sample then Project) and optimized plan (Project then Sample). In the extreme case such as attached in the JIRA, if the projected column is an UDF which is supposed to not see the sampled out data, the result of UDF will be incorrect. Since the rule `ColumnPruning` already handles general `Project` pushdown. We don't need `PushProjectThroughSample` anymore. The rule `ColumnPruning` also avoids the described issue. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14327 from viirya/fix-sample-pushdown.
* [SPARK-16633][SPARK-16642][SPARK-16721][SQL] Fixes three issues related to ↵Yin Huai2016-07-251-0/+414
| | | | | | | | | | | | | | | | | | | | | | | lead and lag functions ## What changes were proposed in this pull request? This PR contains three changes. First, this PR changes the behavior of lead/lag back to Spark 1.6's behavior, which is described as below: 1. lead/lag respect null input values, which means that if the offset row exists and the input value is null, the result will be null instead of the default value. 2. If the offset row does not exist, the default value will be used. 3. OffsetWindowFunction's nullable setting also considers the nullability of its input (because of the first change). Second, this PR fixes the evaluation of lead/lag when the input expression is a literal. This fix is a result of the first change. In current master, if a literal is used as the input expression of a lead or lag function, the result will be this literal even if the offset row does not exist. Third, this PR makes ResolveWindowFrame not fire if a window function is not resolved. ## How was this patch tested? New tests in SQLWindowFunctionSuite Author: Yin Huai <yhuai@databricks.com> Closes #14284 from yhuai/lead-lag.
* [SPARK-14131][STREAMING] SQL Improved fix for avoiding potential deadlocks ↵Tathagata Das2016-07-253-7/+50
| | | | | | | | | | | | | | in HDFSMetadataLog ## What changes were proposed in this pull request? Current fix for deadlock disables interrupts in the StreamExecution which getting offsets for all sources, and when writing to any metadata log, to avoid potential deadlocks in HDFSMetadataLog(see JIRA for more details). However, disabling interrupts can have unintended consequences in other sources. So I am making the fix more narrow, by disabling interrupt it only in the HDFSMetadataLog. This is a narrower fix for something risky like disabling interrupt. ## How was this patch tested? Existing tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14292 from tdas/SPARK-14131.
* [SPARK-16698][SQL] Field names having dots should be allowed for datasources ↵hyukjinkwon2016-07-251-0/+15
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | based on FileFormat ## What changes were proposed in this pull request? It seems this is a regression assuming from https://issues.apache.org/jira/browse/SPARK-16698. Field name having dots throws an exception. For example the codes below: ```scala val path = "/tmp/path" val json =""" {"a.b":"data"}""" spark.sparkContext .parallelize(json :: Nil) .saveAsTextFile(path) spark.read.json(path).collect() ``` throws an exception as below: ``` Unable to resolve a.b given [a.b]; org.apache.spark.sql.AnalysisException: Unable to resolve a.b given [a.b]; at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134) at scala.Option.getOrElse(Option.scala:121) ``` This problem was introduced in https://github.com/apache/spark/commit/17eec0a71ba8713c559d641e3f43a1be726b037c#diff-27c76f96a7b2733ecfd6f46a1716e153R121 When extracting the data columns, it does not count that it can contains dots in field names. Actually, it seems the fields name are not expected as quoted when defining schema. So, It not have to consider whether this is wrapped with quotes because the actual schema (inferred or user-given schema) would not have the quotes for fields. For example, this throws an exception. (**Loading JSON from RDD is fine**) ```scala val json =""" {"a.b":"data"}""" val rdd = spark.sparkContext.parallelize(json :: Nil) spark.read.schema(StructType(Seq(StructField("`a.b`", StringType, true)))) .json(rdd).select("`a.b`").printSchema() ``` as below: ``` cannot resolve '```a.b```' given input columns: [`a.b`]; org.apache.spark.sql.AnalysisException: cannot resolve '```a.b```' given input columns: [`a.b`]; at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42) ``` ## How was this patch tested? Unit tests in `FileSourceStrategySuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14339 from HyukjinKwon/SPARK-16698-regression.
* [SPARK-16668][TEST] Test parquet reader for row groups containing both ↵Sameer Agarwal2016-07-251-0/+29
| | | | | | | | | | | | | | | | | dictionary and plain encoded pages ## What changes were proposed in this pull request? This patch adds an explicit test for [SPARK-14217] by setting the parquet dictionary and page size the generated parquet file spans across 3 pages (within a single row group) where the first page is dictionary encoded and the remaining two are plain encoded. ## How was this patch tested? 1. ParquetEncodingSuite 2. Also manually tested that this test fails without https://github.com/apache/spark/pull/12279 Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #14304 from sameeragarwal/hybrid-encoding-test.
* [SPARK-16691][SQL] move BucketSpec to catalyst module and use it in CatalogTableWenchen Fan2016-07-255-8/+9
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? It's weird that we have `BucketSpec` to abstract bucket info, but don't use it in `CatalogTable`. This PR moves `BucketSpec` into catalyst module. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14331 from cloud-fan/check.
* [SPARK-16645][SQL] rename CatalogStorageFormat.serdeProperties to propertiesWenchen Fan2016-07-251-15/+15
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? we also store data source table options in this field, it's unreasonable to call it `serdeProperties`. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14283 from cloud-fan/minor1.
* [SPARK-16463][SQL] Support `truncate` option in Overwrite mode for JDBC ↵Dongjoon Hyun2016-07-241-2/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | DataFrameWriter ## What changes were proposed in this pull request? This PR adds a boolean option, `truncate`, for `SaveMode.Overwrite` of JDBC DataFrameWriter. If this option is `true`, it try to take advantage of `TRUNCATE TABLE` instead of `DROP TABLE`. This is a trivial option, but will provide great **convenience** for BI tool users based on RDBMS tables generated by Spark. **Goal** - Without `CREATE/DROP` privilege, we can save dataframe to database. Sometime these are not allowed for security. - It will preserve the existing table information, so users can add and keep some additional `INDEX` and `CONSTRAINT`s for the table. - Sometime, `TRUNCATE` is faster than the combination of `DROP/CREATE`. **Supported DBMS** The following is `truncate`-option support table. Due to the different behavior of `TRUNCATE TABLE` among DBMSs, it's not always safe to use `TRUNCATE TABLE`. Spark will ignore the `truncate` option for **unknown** and **some** DBMS with **default CASCADING** behavior. Newly added JDBCDialect should implement corresponding function to support `truncate` option additionally. Spark Dialects | `truncate` OPTION SUPPORT ---------------|------------------------------- MySQLDialect | O PostgresDialect | X DB2Dialect | O MsSqlServerDialect | O DerbyDialect | O OracleDialect | O **Before (TABLE with INDEX case)**: SparkShell & MySQL CLI are interleaved intentionally. ```scala scala> val (url, prop)=("jdbc:mysql://localhost:3306/temp?useSSL=false", new java.util.Properties) scala> prop.setProperty("user","root") scala> df.write.mode("overwrite").jdbc(url, "table_with_index", prop) scala> spark.range(10).write.mode("overwrite").jdbc(url, "table_with_index", prop) mysql> DESC table_with_index; +-------+------------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-------+------------+------+-----+---------+-------+ | id | bigint(20) | NO | | NULL | | +-------+------------+------+-----+---------+-------+ mysql> CREATE UNIQUE INDEX idx_id ON table_with_index(id); mysql> DESC table_with_index; +-------+------------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-------+------------+------+-----+---------+-------+ | id | bigint(20) | NO | PRI | NULL | | +-------+------------+------+-----+---------+-------+ scala> spark.range(10).write.mode("overwrite").jdbc(url, "table_with_index", prop) mysql> DESC table_with_index; +-------+------------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-------+------------+------+-----+---------+-------+ | id | bigint(20) | NO | | NULL | | +-------+------------+------+-----+---------+-------+ ``` **After (TABLE with INDEX case)** ```scala scala> spark.range(10).write.mode("overwrite").option("truncate", true).jdbc(url, "table_with_index", prop) mysql> DESC table_with_index; +-------+------------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-------+------------+------+-----+---------+-------+ | id | bigint(20) | NO | PRI | NULL | | +-------+------------+------+-----+---------+-------+ ``` **Error Handling** - In case of exceptions, Spark will not retry. Users should turn off the `truncate` option. - In case of schema change: - If one of the column names changes, this will raise exceptions intuitively. - If there exists only type difference, this will work like Append mode. ## How was this patch tested? Pass the Jenkins tests with a updated testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14086 from dongjoon-hyun/SPARK-16410.
* [SPARK-16690][TEST] rename SQLTestUtils.withTempTable to withTempViewWenchen Fan2016-07-2313-55/+55
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? after https://github.com/apache/spark/pull/12945, we renamed the `registerTempTable` to `createTempView`, as we do create a view actually. This PR renames `SQLTestUtils.withTempTable` to reflect this change. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14318 from cloud-fan/minor4.
* [SPARK-16556][SPARK-16559][SQL] Fix Two Bugs in Bucket Specificationgatorsmile2016-07-222-2/+43
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? **Issue 1: Silent Ignorance of Bucket Specification When Creating Table Using Schema Inference** When creating a data source table without explicit specification of schema or SELECT clause, we silently ignore the bucket specification (CLUSTERED BY... SORTED BY...) in [the code](https://github.com/apache/spark/blob/ce3b98bae28af72299722f56e4e4ef831f471ec0/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala#L339-L354). For example, ```SQL CREATE TABLE jsonTable USING org.apache.spark.sql.json OPTIONS ( path '${tempDir.getCanonicalPath}' ) CLUSTERED BY (inexistentColumnA) SORTED BY (inexistentColumnB) INTO 2 BUCKETS ``` This PR captures it and issues an error message. **Issue 2: Got a run-time `java.lang.ArithmeticException` when num of buckets is set to zero.** For example, ```SQL CREATE TABLE t USING PARQUET OPTIONS (PATH '${path.toString}') CLUSTERED BY (a) SORTED BY (b) INTO 0 BUCKETS AS SELECT 1 AS a, 2 AS b ``` The exception we got is ``` ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 1.0 (TID 2) java.lang.ArithmeticException: / by zero ``` This PR captures the misuse and issues an appropriate error message. ### How was this patch tested? Added a test case in DDLSuite Author: gatorsmile <gatorsmile@gmail.com> Closes #14210 from gatorsmile/createTableWithoutSchema.
* [SPARK-16287][SQL] Implement str_to_map SQL functionSandeep Singh2016-07-221-0/+23
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds `str_to_map` SQL function in order to remove Hive fallback. ## How was this patch tested? Pass the Jenkins tests with newly added. Author: Sandeep Singh <sandeep@techaddict.me> Closes #13990 from techaddict/SPARK-16287.
* [SPARK-16656][SQL] Try to make CreateTableAsSelectSuite more stableYin Huai2016-07-211-10/+15
| | | | | | | | | ## What changes were proposed in this pull request? https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62593/testReport/junit/org.apache.spark.sql.sources/CreateTableAsSelectSuite/create_a_table__drop_it_and_create_another_one_with_the_same_name/ shows that `create a table, drop it and create another one with the same name` failed. But other runs were good. Seems it is a flaky test. This PR tries to make this test more stable. Author: Yin Huai <yhuai@databricks.com> Closes #14289 from yhuai/SPARK-16656.
* [SPARK-16632][SQL] Revert PR #14272: Respect Hive schema when merging ↵Cheng Lian2016-07-211-39/+0
| | | | | | | | | | | | | | | | | | parquet schema ## What changes were proposed in this pull request? PR #14278 is a more general and simpler fix for SPARK-16632 than PR #14272. After merging #14278, we no longer need changes made in #14272. So here I revert them. This PR targets both master and branch-2.0. ## How was this patch tested? Existing tests. Author: Cheng Lian <lian@databricks.com> Closes #14300 from liancheng/revert-pr-14272.
* [SPARK-16632][SQL] Use Spark requested schema to guide vectorized Parquet ↵Cheng Lian2016-07-211-0/+24
| | | | | | | | | | | | | | | | | | | | reader initialization ## What changes were proposed in this pull request? In `SpecificParquetRecordReaderBase`, which is used by the vectorized Parquet reader, we convert the Parquet requested schema into a Spark schema to guide column reader initialization. However, the Parquet requested schema is tailored from the schema of the physical file being scanned, and may have inaccurate type information due to bugs of other systems (e.g. HIVE-14294). On the other hand, we already set the real Spark requested schema into Hadoop configuration in [`ParquetFileFormat`][1]. This PR simply reads out this schema to replace the converted one. ## How was this patch tested? New test case added in `ParquetQuerySuite`. [1]: https://github.com/apache/spark/blob/v2.0.0-rc5/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L292-L294 Author: Cheng Lian <lian@databricks.com> Closes #14278 from liancheng/spark-16632-simpler-fix.
* [SPARK-16644][SQL] Aggregate should not propagate constraints containing ↵Wenchen Fan2016-07-201-0/+17
| | | | | | | | | | | | | | | | | aggregate expressions ## What changes were proposed in this pull request? aggregate expressions can only be executed inside `Aggregate`, if we propagate it up with constraints, the parent operator can not execute it and will fail at runtime. ## How was this patch tested? new test in SQLQuerySuite Author: Wenchen Fan <wenchen@databricks.com> Author: Yin Huai <yhuai@databricks.com> Closes #14281 from cloud-fan/bug.
* [SPARK-16344][SQL] Decoding Parquet array of struct with a single field ↵Cheng Lian2016-07-201-1/+15
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | named "element" ## What changes were proposed in this pull request? Due to backward-compatibility reasons, the following Parquet schema is ambiguous: ``` optional group f (LIST) { repeated group list { optional group element { optional int32 element; } } } ``` According to the parquet-format spec, when interpreted as a standard 3-level layout, this type is equivalent to the following SQL type: ``` ARRAY<STRUCT<element: INT>> ``` However, when interpreted as a legacy 2-level layout, it's equivalent to ``` ARRAY<STRUCT<element: STRUCT<element: INT>>> ``` Historically, to disambiguate these cases, we employed two methods: - `ParquetSchemaConverter.isElementType()` Used to disambiguate the above cases while converting Parquet types to Spark types. - `ParquetRowConverter.isElementType()` Used to disambiguate the above cases while instantiating row converters that convert Parquet records to Spark rows. Unfortunately, these two methods make different decision about the above problematic Parquet type, and caused SPARK-16344. `ParquetRowConverter.isElementType()` is necessary for Spark 1.4 and earlier versions because Parquet requested schemata are directly converted from Spark schemata in these versions. The converted Parquet schemata may be incompatible with actual schemata of the underlying physical files when the files are written by a system/library that uses a schema conversion scheme that is different from Spark when writing Parquet LIST and MAP fields. In Spark 1.5, Parquet requested schemata are always properly tailored from schemata of physical files to be read. Thus `ParquetRowConverter.isElementType()` is no longer necessary. This PR replaces this method with a simply yet accurate scheme: whenever an ambiguous Parquet type is hit, convert the type in question back to a Spark type using `ParquetSchemaConverter` and check whether it matches the corresponding Spark type. ## How was this patch tested? New test cases added in `ParquetHiveCompatibilitySuite` and `ParquetQuerySuite`. Author: Cheng Lian <lian@databricks.com> Closes #14014 from liancheng/spark-16344-for-master-and-2.0.
* [SPARK-16632][SQL] Respect Hive schema when merging parquet schema.Marcelo Vanzin2016-07-201-0/+39
| | | | | | | | | | | | | | | | | | | | | | | When Hive (or at least certain versions of Hive) creates parquet files containing tinyint or smallint columns, it stores them as int32, but doesn't annotate the parquet field as containing the corresponding int8 / int16 data. When Spark reads those files using the vectorized reader, it follows the parquet schema for these fields, but when actually reading the data it tries to use the type fetched from the metastore, and then fails because data has been loaded into the wrong fields in OnHeapColumnVector. So instead of blindly trusting the parquet schema, check whether the Catalyst-provided schema disagrees with it, and adjust the types so that the necessary metadata is present when loading the data into the ColumnVector instance. Tested with unit tests and with tests that create byte / short columns in Hive and try to read them from Spark. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14272 from vanzin/SPARK-16632.
* [SPARK-16351][SQL] Avoid per-record type dispatch in JSON when writinghyukjinkwon2016-07-181-3/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `JacksonGenerator.apply` is doing type-based dispatch for each row to write appropriate values. It might not have to be done like this because the schema is already kept. So, appropriate writers can be created first according to the schema once, and then apply them to each row. This approach is similar with `CatalystWriteSupport`. This PR corrects `JacksonGenerator` so that it creates all writers for the schema once and then applies them to each row rather than type dispatching for every row. Benchmark was proceeded with the codes below: ```scala test("Benchmark for JSON writer") { val N = 500 << 8 val row = """{"struct":{"field1": true, "field2": 92233720368547758070}, "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, "arrayOfString":["str1", "str2"], "arrayOfInteger":[1, 2147483647, -2147483648], "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" val df = spark.sqlContext.read.json(spark.sparkContext.parallelize(List.fill(N)(row))) val benchmark = new Benchmark("JSON writer", N) benchmark.addCase("writing JSON file", 10) { _ => withTempPath { path => df.write.format("json").save(path.getCanonicalPath) } } benchmark.run() } ``` This produced the results below - **Before** ``` JSON writer: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ writing JSON file 1675 / 1767 0.1 13087.5 1.0X ``` - **After** ``` JSON writer: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ writing JSON file 1597 / 1686 0.1 12477.1 1.0X ``` In addition, I ran this benchmark 10 times for each and calculated the average elapsed time as below: | **Before** | **After**| |---------------|------------| |17478ms |16669ms | It seems roughly ~5% is improved. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14028 from HyukjinKwon/SPARK-16351.
* [SPARK-16588][SQL] Deprecate monotonicallyIncreasingId in Scala/JavaReynold Xin2016-07-171-7/+6
| | | | | | This patch deprecates monotonicallyIncreasingId in Scala/Java, as done in Python. This patch was originally written by HyukjinKwon. Closes #14236.
* [SPARK-16529][SQL][TEST] `withTempDatabase` should set `default` database ↵Dongjoon Hyun2016-07-151-1/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | before dropping ## What changes were proposed in this pull request? `SQLTestUtils.withTempDatabase` is a frequently used test harness to setup a temporary table and clean up finally. This issue improves like the following for usability. ```scala - try f(dbName) finally spark.sql(s"DROP DATABASE $dbName CASCADE") + try f(dbName) finally { + if (spark.catalog.currentDatabase == dbName) { + spark.sql(s"USE ${DEFAULT_DATABASE}") + } + spark.sql(s"DROP DATABASE $dbName CASCADE") + } ``` In case of forgetting to reset the databaes, `withTempDatabase` will not raise Exception. ## How was this patch tested? This improves test harness. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14184 from dongjoon-hyun/SPARK-16529.
* [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.
* [MINOR] Fix Java style errors and remove unused importsXin Ren2016-07-131-1/+0
| | | | | | | | | | | | | | ## 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-125-32/+43
| | | | | | | | | | | | | | | | | | | | | | | | | 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-122-0/+156
| | | | | | | | | | | | | | | | 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-121-1/+7
| | | | | | | | | | | | | | | | | | | 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-111-0/+89
| | | | | | | | | | | | | | | | | 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-111-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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-111-2/+33
| | | | | | | | | | | | | | | | 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-112-4/+4
| | | | | | | | | | | | | | ## 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-111-1/+51
| | | | | | | | | | | | | | ## 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-111-0/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-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-091-0/+32
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-081-0/+6
| | | | | | | | | | | | | | | | | | | | | | | | ## 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-081-18/+18
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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-081-0/+26
| | | | | | | | | | | | | | | ## 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-071-0/+76
| | | | | | | | | | | | | | | | ## 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-071-9/+77
| | | | | | | | | | | | | | | | | | ## 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-071-30/+0
| | | | | | | | | | | | ## 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-071-0/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | | | Column Num #### What changes were proposed in this pull request? When creating a view, a common user error is the number of columns produced by the `SELECT` clause does not match the number of column names specified by `CREATE VIEW`. For example, given Table `t1` only has 3 columns ```SQL create view v1(col2, col4, col3, col5) as select * from t1 ``` Currently, Spark SQL reports the following error: ``` requirement failed java.lang.IllegalArgumentException: requirement failed at scala.Predef$.require(Predef.scala:212) at org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:90) ``` This error message is very confusing. This PR is to detect the error and issue a meaningful error message. #### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14047 from gatorsmile/viewMismatchedColumns.
* [SPARK-14839][SQL] Support for other types for `tableProperty` rule in SQL ↵hyukjinkwon2016-07-061-0/+61
| | | | | | | | | | | | | | | | | | | | 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.