aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-16664][SQL] Fix persist call on Data frames with more than 200…Wesley Tang2016-07-293-3/+12
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? f12f11e578169b47e3f8b18b299948c0670ba585 introduced this bug, missed foreach as map ## How was this patch tested? Test added Author: Wesley Tang <tangmingjun@mininglamp.com> Closes #14324 from breakdawn/master.
* [SPARK-16764][SQL] Recommend disabling vectorized parquet reader on ↵Sameer Agarwal2016-07-281-5/+19
| | | | | | | | | | | | | | | | OutOfMemoryError ## What changes were proposed in this pull request? We currently don't bound or manage the data array size used by column vectors in the vectorized reader (they're just bound by INT.MAX) which may lead to OOMs while reading data. As a short term fix, this patch intercepts the OutOfMemoryError exception and suggest the user to disable the vectorized parquet reader. ## How was this patch tested? Existing Tests Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #14387 from sameeragarwal/oom.
* [SPARK-16740][SQL] Fix Long overflow in LongToUnsafeRowMapSylvain Zimmer2016-07-281-1/+2
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Avoid overflow of Long type causing a NegativeArraySizeException a few lines later. ## How was this patch tested? Unit tests for HashedRelationSuite still pass. I can confirm the python script I included in https://issues.apache.org/jira/browse/SPARK-16740 works fine with this patch. Unfortunately I don't have the knowledge/time to write a Scala test case for HashedRelationSuite right now. As the patch is pretty obvious I hope it can be included without this. Thanks! Author: Sylvain Zimmer <sylvain@sylvainzimmer.com> Closes #14373 from sylvinus/master.
* [SPARK-16639][SQL] The query with having condition that contains grouping by ↵Liang-Chi Hsieh2016-07-282-6/+31
| | | | | | | | | | | | | | | | | | | | | column should work ## What changes were proposed in this pull request? The query with having condition that contains grouping by column will be failed during analysis. E.g., create table tbl(a int, b string); select count(b) from tbl group by a + 1 having a + 1 = 2; Having condition should be able to use grouping by column. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14296 from viirya/having-contains-grouping-column.
* [SPARK-16552][SQL] Store the Inferred Schemas into External Catalog Tables ↵gatorsmile2016-07-287-79/+291
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | when Creating Tables #### What changes were proposed in this pull request? Currently, in Spark SQL, the initial creation of schema can be classified into two groups. It is applicable to both Hive tables and Data Source tables: **Group A. Users specify the schema.** _Case 1 CREATE TABLE AS SELECT_: the schema is determined by the result schema of the SELECT clause. For example, ```SQL CREATE TABLE tab STORED AS TEXTFILE AS SELECT * from input ``` _Case 2 CREATE TABLE_: users explicitly specify the schema. For example, ```SQL CREATE TABLE jsonTable (_1 string, _2 string) USING org.apache.spark.sql.json ``` **Group B. Spark SQL infers the schema at runtime.** _Case 3 CREATE TABLE_. Users do not specify the schema but the path to the file location. For example, ```SQL CREATE TABLE jsonTable USING org.apache.spark.sql.json OPTIONS (path '${tempDir.getCanonicalPath}') ``` Before this PR, Spark SQL does not store the inferred schema in the external catalog for the cases in Group B. When users refreshing the metadata cache, accessing the table at the first time after (re-)starting Spark, Spark SQL will infer the schema and store the info in the metadata cache for improving the performance of subsequent metadata requests. However, the runtime schema inference could cause undesirable schema changes after each reboot of Spark. This PR is to store the inferred schema in the external catalog when creating the table. When users intend to refresh the schema after possible changes on external files (table location), they issue `REFRESH TABLE`. Spark SQL will infer the schema again based on the previously specified table location and update/refresh the schema in the external catalog and metadata cache. In this PR, we do not use the inferred schema to replace the user specified schema for avoiding external behavior changes . Based on the design, user-specified schemas (as described in Group A) can be changed by ALTER TABLE commands, although we do not support them now. #### How was this patch tested? TODO: add more cases to cover the changes. Author: gatorsmile <gatorsmile@gmail.com> Closes #14207 from gatorsmile/userSpecifiedSchema.
* [SPARK-15232][SQL] Add subquery SQL building tests to LogicalPlanToSQLSuiteDongjoon Hyun2016-07-2715-0/+282
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We currently test subquery SQL building using the `HiveCompatibilitySuite`. The is not desired since SQL building is actually a part of `sql/core` and because we are slowly reducing our dependency on Hive. This PR adds the same tests from the whitelist of `HiveCompatibilitySuite` into `LogicalPlanToSQLSuite`. ## How was this patch tested? This adds more testcases. Pass the Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14383 from dongjoon-hyun/SPARK-15232.
* [SPARK-16730][SQL] Implement function aliases for type castspetermaxlee2016-07-283-7/+73
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Spark 1.x supports using the Hive type name as function names for doing casts, e.g. ```sql SELECT int(1.0); SELECT string(2.0); ``` The above query would work in Spark 1.x because Spark 1.x fail back to Hive for unimplemented functions, and break in Spark 2.0 because the fall back was removed. This patch implements function aliases using an analyzer rule for the following cast functions: - boolean - tinyint - smallint - int - bigint - float - double - decimal - date - timestamp - binary - string ## How was this patch tested? Added end-to-end tests in SQLCompatibilityFunctionSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14364 from petermaxlee/SPARK-16730-2.
* [MINOR][DOC][SQL] Fix two documents regarding size in bytesLiang-Chi Hsieh2016-07-271-5/+7
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix two places in SQLConf documents regarding size in bytes and statistics. ## How was this patch tested? No. Just change document. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14341 from viirya/fix-doc-size-in-bytes.
* [SPARK-16729][SQL] Throw analysis exception for invalid date castspetermaxlee2016-07-272-9/+16
| | | | | | | | | | | | ## What changes were proposed in this pull request? Spark currently throws exceptions for invalid casts for all other data types except date type. Somehow date type returns null. It should be consistent and throws analysis exception as well. ## How was this patch tested? Added a unit test case in CastSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14358 from petermaxlee/SPARK-16729.
* [SPARK-16621][SQL] Generate stable SQLs in SQLBuilderDongjoon Hyun2016-07-27105-114/+124
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, the generated SQLs have not-stable IDs for generated attributes. The stable generated SQL will give more benefit for understanding or testing the queries. This PR provides stable SQL generation by the followings. - Provide unique ids for generated subqueries, `gen_subquery_xxx`. - Provide unique and stable ids for generated attributes, `gen_attr_xxx`. **Before** ```scala scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res0: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res1: String = SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4`) AS gen_subquery_0 ``` **After** ```scala scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res1: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res2: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 ``` ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14257 from dongjoon-hyun/SPARK-16621.
* [SPARK-16524][SQL] Add RowBatch and RowBasedHashMapGeneratorQifan Pu2016-07-267-126/+1356
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is the first step for the following feature: For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields). In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBasedKeyValueBatch`. We then automatically pick between the two implementations based on certain knobs. In this first-step PR, implementations for `RowBasedKeyValueBatch` and `RowBasedHashMapGenerator` are added. ## How was this patch tested? Unit tests: `RowBasedKeyValueBatchSuite` Author: Qifan Pu <qifan.pu@gmail.com> Closes #14349 from ooq/SPARK-16524.
* [SPARK-16663][SQL] desc table should be consistent between data source and ↵Wenchen Fan2016-07-265-31/+47
| | | | | | | | | | | | | | | | | | | hive serde tables ## What changes were proposed in this pull request? Currently there are 2 inconsistence: 1. for data source table, we only print partition names, for hive table, we also print partition schema. After this PR, we will always print schema 2. if column doesn't have comment, data source table will print empty string, hive table will print null. After this PR, we will always print null ## How was this patch tested? new test in `HiveDDLSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14302 from cloud-fan/minor3.
* [SPARK-16675][SQL] Avoid per-record type dispatch in JDBC when writinghyukjinkwon2016-07-262-36/+88
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `JdbcUtils.savePartition` is doing type-based dispatch for each row to write appropriate values. So, appropriate setters for `PreparedStatement` can be created first according to the schema, and then apply them to each row. This approach is similar with `CatalystWriteSupport`. This PR simply make the setters to avoid this. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14323 from HyukjinKwon/SPARK-16675.
* [SPARK-16706][SQL] support java map in encoderWenchen Fan2016-07-265-32/+236
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? finish the TODO, create a new expression `ExternalMapToCatalyst` to iterate the map directly. ## How was this patch tested? new test in `JavaDatasetSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14344 from cloud-fan/java-map.
* [SPARK-16686][SQL] Remove PushProjectThroughSample since it is handled by ↵Liang-Chi Hsieh2016-07-264-29/+40
| | | | | | | | | | | | | | | | | | 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-254-40/+108
| | | | | | | | | | | | | | | | | | | | | | | 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-16724] Expose DefinedByConstructorParamsMichael Armbrust2016-07-251-1/+1
| | | | | | | | We don't generally make things in catalyst/execution private. Instead they are just undocumented due to their lack of stability guarantees. Author: Michael Armbrust <michael@databricks.com> Closes #14356 from marmbrus/patch-1.
* [SPARK-16672][SQL] SQLBuilder should not raise exceptions on EXISTS queriesDongjoon Hyun2016-07-253-2/+21
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `SQLBuilder` raises `empty.reduceLeft` exceptions on *unoptimized* `EXISTS` queries. We had better prevent this. ```scala scala> sql("CREATE TABLE t1(a int)") scala> val df = sql("select * from t1 b where exists (select * from t1 a)") scala> new org.apache.spark.sql.catalyst.SQLBuilder(df).toSQL java.lang.UnsupportedOperationException: empty.reduceLeft ``` ## How was this patch tested? Pass the Jenkins tests with a new test suite. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14307 from dongjoon-hyun/SPARK-16672.
* [SPARK-16678][SPARK-16677][SQL] Fix two View-related bugsgatorsmile2016-07-264-19/+96
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? **Issue 1: Disallow Creating/Altering a View when the same-name Table Exists (without IF NOT EXISTS)** When we create OR alter a view, we check whether the view already exists. In the current implementation, if a table with the same name exists, we treat it as a view. However, this is not the right behavior. We should follow what Hive does. For example, ``` hive> CREATE TABLE tab1 (id int); OK Time taken: 0.196 seconds hive> CREATE OR REPLACE VIEW tab1 AS SELECT * FROM t1; FAILED: SemanticException [Error 10218]: Existing table is not a view The following is an existing table, not a view: default.tab1 hive> ALTER VIEW tab1 AS SELECT * FROM t1; FAILED: SemanticException [Error 10218]: Existing table is not a view The following is an existing table, not a view: default.tab1 hive> CREATE VIEW IF NOT EXISTS tab1 AS SELECT * FROM t1; OK Time taken: 0.678 seconds ``` **Issue 2: Strange Error when Issuing Load Table Against A View** Users should not be allowed to issue LOAD DATA against a view. Currently, when users doing it, we got a very strange runtime error. For example, ```SQL LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName ``` ``` java.lang.reflect.InvocationTargetException was thrown. java.lang.reflect.InvocationTargetException at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.sql.hive.client.Shim_v0_14.loadTable(HiveShim.scala:680) ``` ## How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14314 from gatorsmile/tableDDLAgainstView.
* [SPARK-14131][STREAMING] SQL Improved fix for avoiding potential deadlocks ↵Tathagata Das2016-07-255-36/+80
| | | | | | | | | | | | | | 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-16715][TESTS] Fix a potential ExprId conflict for ↵Shixiong Zhu2016-07-251-1/+5
| | | | | | | | | | | | | | | | | | SubexpressionEliminationSuite."Semantic equals and hash" ## What changes were proposed in this pull request? SubexpressionEliminationSuite."Semantic equals and hash" assumes the default AttributeReference's exprId wont' be "ExprId(1)". However, that depends on when this test runs. It may happen to use "ExprId(1)". This PR detects the conflict and makes sure we create a different ExprId when the conflict happens. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #14350 from zsxwing/SPARK-16715.
* [SPARK-16703][SQL] Remove extra whitespace in SQL generation for window ↵Cheng Lian2016-07-256-9/+40
| | | | | | | | | | | | | | | | | | | | | | | | | | | | functions ## What changes were proposed in this pull request? This PR fixes a minor formatting issue of `WindowSpecDefinition.sql` when no partitioning expressions are present. Before: ```sql ( ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) ``` After: ```sql (ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) ``` ## How was this patch tested? New test case added in `ExpressionSQLBuilderSuite`. Author: Cheng Lian <lian@databricks.com> Closes #14334 from liancheng/window-spec-sql-format.
* [SPARK-16698][SQL] Field names having dots should be allowed for datasources ↵hyukjinkwon2016-07-252-1/+16
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-2521-78/+78
| | | | | | | | | | | | | | ## 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-16660][SQL] CreateViewCommand should not take CatalogTableWenchen Fan2016-07-256-127/+116
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `CreateViewCommand` only needs some information of a `CatalogTable`, but not all of them. We have some tricks(e.g. we need to check the table type is `VIEW`, we need to make `CatalogColumn.dataType` nullable) to allow it to take a `CatalogTable`. This PR cleans it up and only pass in necessary information to `CreateViewCommand`. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14297 from cloud-fan/minor2.
* [SPARK-16674][SQL] Avoid per-record type dispatch in JDBC when readinghyukjinkwon2016-07-251-116/+129
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `JDBCRDD.compute` is doing type dispatch for each row to read appropriate values. It might not have to be done like this because the schema is already kept in `JDBCRDD`. So, appropriate converters can be created first according to the schema, and then apply them to each row. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14313 from HyukjinKwon/SPARK-16674.
* [SPARK-16648][SQL] Make ignoreNullsExpr a child expression of First and LastCheng Lian2016-07-253-4/+16
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Default `TreeNode.withNewChildren` implementation doesn't work for `Last` and when both constructor arguments are the same, e.g.: ```sql LAST_VALUE(FALSE) -- The 2nd argument defaults to FALSE LAST_VALUE(FALSE, FALSE) LAST_VALUE(TRUE, TRUE) ``` This is because although `Last` is a unary expression, both of its constructor arguments, `child` and `ignoreNullsExpr`, are `Expression`s. When they have the same value, `TreeNode.withNewChildren` treats both of them as child nodes by mistake. `First` is also affected by this issue in exactly the same way. This PR fixes this issue by making `ignoreNullsExpr` a child expression of `First` and `Last`. ## How was this patch tested? New test case added in `WindowQuerySuite`. Author: Cheng Lian <lian@databricks.com> Closes #14295 from liancheng/spark-16648-last-value.
* [SPARK-16699][SQL] Fix performance bug in hash aggregate on long string keysQifan Pu2016-07-241-2/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | In the following code in `VectorizedHashMapGenerator.scala`: ``` def hashBytes(b: String): String = { val hash = ctx.freshName("hash") s""" |int $result = 0; |for (int i = 0; i < $b.length; i++) { | ${genComputeHash(ctx, s"$b[i]", ByteType, hash)} | $result = ($result ^ (0x9e3779b9)) + $hash + ($result << 6) + ($result >>> 2); |} """.stripMargin } ``` when b=input.getBytes(), the current 2.0 code results in getBytes() being called n times, n being length of input. getBytes() involves memory copy is thus expensive and causes a performance degradation. Fix is to evaluate getBytes() before the for loop. Performance bug, no additional test added. Author: Qifan Pu <qifan.pu@gmail.com> Closes #14337 from ooq/SPARK-16699. (cherry picked from commit d226dce12babcd9f30db033417b2b9ce79f44312) Signed-off-by: Reynold Xin <rxin@databricks.com>
* [SPARK-16645][SQL] rename CatalogStorageFormat.serdeProperties to propertiesWenchen Fan2016-07-2516-65/+65
| | | | | | | | | | | | | | ## 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-249-4/+70
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-16515][SQL][FOLLOW-UP] Fix test `script` on OS X/Windows...Liwei Lin2016-07-242-9/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## Problem The current `sed` in `test_script.sh` is missing a `$`, leading to the failure of `script` test on OS X: ``` == Results == !== Correct Answer - 2 == == Spark Answer - 2 == ![x1_y1] [x1] ![x2_y2] [x2] ``` In addition, this `script` test would also fail on systems like Windows where we couldn't be able to invoke `bash` or `echo | sed`. ## What changes were proposed in this pull request? This patch - fixes `sed` in `test_script.sh` - adds command guards so that the `script` test would pass on systems like Windows ## How was this patch tested? - Jenkins - Manually verified tests pass on OS X Author: Liwei Lin <lwlin7@gmail.com> Closes #14280 from lw-lin/osx-sed.
* [SPARK-16690][TEST] rename SQLTestUtils.withTempTable to withTempViewWenchen Fan2016-07-2328-89/+89
| | | | | | | | | | | | | | ## 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-16622][SQL] Fix NullPointerException when the returned value of the ↵Liang-Chi Hsieh2016-07-232-17/+65
| | | | | | | | | | | | | | | | called method in Invoke is null ## What changes were proposed in this pull request? Currently we don't check the value returned by called method in `Invoke`. When the returned value is null and is assigned to a variable of primitive type, `NullPointerException` will be thrown. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14259 from viirya/agg-empty-ds.
* [SPARK-16287][HOTFIX][BUILD][SQL] Fix annotation argument needs to be a constantJacek Laskowski2016-07-221-3/+3
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Build fix for [SPARK-16287][SQL] Implement str_to_map SQL function that has introduced this compilation error: ``` /Users/jacek/dev/oss/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala:402: error: annotation argument needs to be a constant; found: "_FUNC_(text[, pairDelim, keyValueDelim]) - Creates a map after splitting the text ".+("into key/value pairs using delimiters. ").+("Default delimiters are \',\' for pairDelim and \':\' for keyValueDelim.") "into key/value pairs using delimiters. " + ^ ``` ## How was this patch tested? Local build Author: Jacek Laskowski <jacek@japila.pl> Closes #14315 from jaceklaskowski/build-fix-complexTypeCreator.
* [SPARK-16556][SPARK-16559][SQL] Fix Two Bugs in Bucket Specificationgatorsmile2016-07-224-3/+55
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### 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-225-3/+112
| | | | | | | | | | | | ## 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-16334] Maintain single dictionary per row-batch in vectorized parquet ↵Sameer Agarwal2016-07-211-8/+13
| | | | | | | | | | | | | | | | reader ## What changes were proposed in this pull request? As part of the bugfix in https://github.com/apache/spark/pull/12279, if a row batch consist of both dictionary encoded and non-dictionary encoded pages, we explicitly decode the dictionary for the values that are already dictionary encoded. Currently we reset the dictionary while reading every page that can potentially cause ` java.lang.ArrayIndexOutOfBoundsException` while decoding older pages. This patch fixes the problem by maintaining a single dictionary per row-batch in vectorized parquet reader. ## How was this patch tested? Manual Tests against a number of hand-generated parquet files. Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #14225 from sameeragarwal/vectorized.
* [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-212-57/+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-16640][SQL] Add codegen for Elt functionLiang-Chi Hsieh2016-07-211-1/+24
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Elt function doesn't support codegen execution now. We should add the support. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14277 from viirya/elt-codegen.
* [SPARK-16632][SQL] Use Spark requested schema to guide vectorized Parquet ↵Cheng Lian2016-07-212-1/+28
| | | | | | | | | | | | | | | | | | | | 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-16226][SQL] Weaken JDBC isolation level to avoid locking when writing ↵Sean Owen2016-07-212-14/+46
| | | | | | | | | | | | | | | | partitions ## What changes were proposed in this pull request? Saving partitions to JDBC in transaction can use a weaker transaction isolation level to reduce locking. Use better method to check if transactions are supported. ## How was this patch tested? Existing Jenkins tests. Author: Sean Owen <sowen@cloudera.com> Closes #14054 from srowen/SPARK-16226.
* [SPARK-16644][SQL] Aggregate should not propagate constraints containing ↵Wenchen Fan2016-07-203-4/+25
| | | | | | | | | | | | | | | | | 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-16272][CORE] Allow config values to reference conf, env, system props.Marcelo Vanzin2016-07-201-6/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | This allows configuration to be more flexible, for example, when the cluster does not have a homogeneous configuration (e.g. packages are installed on different paths in different nodes). By allowing one to reference the environment from the conf, it becomes possible to work around those in certain cases. As part of the implementation, ConfigEntry now keeps track of all "known" configs (i.e. those created through the use of ConfigBuilder), since that list is used by the resolution code. This duplicates some code in SQLConf, which could potentially be merged with this now. It will also make it simpler to implement some missing features such as filtering which configs show up in the UI or in event logs - which are not part of this change. Another change is in the way ConfigEntry reads config data; it now takes a string map and a function that reads env variables, so that it can be called both from SparkConf and SQLConf. This makes it so both places follow the same read path, instead of having to replicate certain logic in SQLConf. There are still a couple of methods in SQLConf that peek into fields of ConfigEntry directly, though. Tested via unit tests, and by using the new variable expansion functionality in a shell session with a custom spark.sql.hive.metastore.jars value. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14022 from vanzin/SPARK-16272.
* [SPARK-16344][SQL] Decoding Parquet array of struct with a single field ↵Cheng Lian2016-07-206-47/+73
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-16634][SQL] Workaround JVM bug by moving some code out of ctor.Marcelo Vanzin2016-07-201-4/+11
| | | | | | | | | | | | | Some 1.7 JVMs have a bug that is triggered by certain Scala-generated bytecode. GenericArrayData suffers from that and fails to load in certain JVMs. Moving the offending code out of the constructor and into a helper method avoids the issue. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14271 from vanzin/SPARK-16634.
* [SPARK-16632][SQL] Respect Hive schema when merging parquet schema.Marcelo Vanzin2016-07-202-0/+57
| | | | | | | | | | | | | | | | | | | | | | | 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-15705][SQL] Change the default value of ↵Yin Huai2016-07-191-1/+2
| | | | | | | | | | | spark.sql.hive.convertMetastoreOrc to false. ## What changes were proposed in this pull request? In 2.0, we add a new logic to convert HiveTableScan on ORC tables to Spark's native code path. However, during this conversion, we drop the original metastore schema (https://issues.apache.org/jira/browse/SPARK-15705). Because of this regression, I am changing the default value of `spark.sql.hive.convertMetastoreOrc` to false. Author: Yin Huai <yhuai@databricks.com> Closes #14267 from yhuai/SPARK-15705-changeDefaultValue.
* [SPARK-16602][SQL] `Nvl` function should support numeric-string casesDongjoon Hyun2016-07-193-2/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `Nvl` function should support numeric-straing cases like Hive/Spark1.6. Currently, `Nvl` finds the tightest common types among numeric types. This PR extends that to consider `String` type, too. ```scala - TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeToString(left.dataType, right.dataType).map { dtype => ``` **Before** ```scala scala> sql("select nvl('0', 1)").collect() org.apache.spark.sql.AnalysisException: cannot resolve `nvl("0", 1)` due to data type mismatch: input to function coalesce should all be the same type, but it's [string, int]; line 1 pos 7 ``` **After** ```scala scala> sql("select nvl('0', 1)").collect() res0: Array[org.apache.spark.sql.Row] = Array([0]) ``` ## How was this patch tested? Pass the Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14251 from dongjoon-hyun/SPARK-16602.