## What changes were proposed in this pull request?
This a follow-up of https://github.com/apache/spark/pull/19289 , we missed another place: `rollup`. `Seq.init.toSeq` also returns a `Stream`, we should fix it too.
## How was this patch tested?
manually
Author: Wenchen Fan <wenchen@databricks.com>
Closes#19298 from cloud-fan/bug.
## What changes were proposed in this pull request?
Spark with Scala 2.10 fails with a group by cube:
```
spark.range(1).select($"id" as "a", $"id" as "b").write.partitionBy("a").mode("overwrite").saveAsTable("rollup_bug")
spark.sql("select 1 from rollup_bug group by rollup ()").show
```
It can be traced back to https://github.com/apache/spark/pull/15484 , which made `Expand.projections` a lazy `Stream` for group by cube.
In scala 2.10 `Stream` captures a lot of stuff, and in this case it captures the entire query plan which has some un-serializable parts.
This change is also good for master branch, to reduce the serialized size of `Expand.projections`.
## How was this patch tested?
manually verified with Spark with Scala 2.10.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#19289 from cloud-fan/bug.
## What changes were proposed in this pull request?
Clarify behavior of to_utc_timestamp/from_utc_timestamp with an example
## How was this patch tested?
Doc only change / existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#19276 from srowen/SPARK-22049.
## What changes were proposed in this pull request?
Tables in the catalog cache are not invalidated once their statistics are updated. As a consequence, existing sessions will use the cached information even though it is not valid anymore. Consider and an example below.
```
// step 1
spark.range(100).write.saveAsTable("tab1")
// step 2
spark.sql("analyze table tab1 compute statistics")
// step 3
spark.sql("explain cost select distinct * from tab1").show(false)
// step 4
spark.range(100).write.mode("append").saveAsTable("tab1")
// step 5
spark.sql("explain cost select distinct * from tab1").show(false)
```
After step 3, the table will be present in the catalog relation cache. Step 4 will correctly update the metadata inside the catalog but will NOT invalidate the cache.
By the way, ``spark.sql("analyze table tab1 compute statistics")`` between step 3 and step 4 would also solve the problem.
## How was this patch tested?
Current and additional unit tests.
Author: aokolnychyi <anton.okolnychyi@sap.com>
Closes#19252 from aokolnychyi/spark-21969.
## What changes were proposed in this pull request?
* Removed the method `org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter#alignToWords`.
It became unused as a result of 85b0a15754
(SPARK-15962) introducing word alignment for unsafe arrays.
* Cleaned up duplicate code in memory management and unsafe sorters
* The change extracting the exception paths is more than just cosmetics since it def. reduces the size the affected methods compile to
## How was this patch tested?
* Build still passes after removing the method, grepping the codebase for `alignToWords` shows no reference to it anywhere either.
* Dried up code is covered by existing tests.
Author: Armin <me@obrown.io>
Closes#19254 from original-brownbear/cleanup-mem-consumer.
#### What changes were proposed in this pull request?
This PR enhances the TRIM function support in Spark SQL by allowing the specification
of trim characters set. Below is the SQL syntax :
``` SQL
<trim function> ::= TRIM <left paren> <trim operands> <right paren>
<trim operands> ::= [ [ <trim specification> ] [ <trim character set> ] FROM ] <trim source>
<trim source> ::= <character value expression>
<trim specification> ::=
LEADING
| TRAILING
| BOTH
<trim character set> ::= <characters value expression>
```
or
``` SQL
LTRIM (source-exp [, trim-exp])
RTRIM (source-exp [, trim-exp])
```
Here are the documentation link of support of this feature by other mainstream databases.
- **Oracle:** [TRIM function](http://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2126.htm#OLADM704)
- **DB2:** [TRIM scalar function](https://www.ibm.com/support/knowledgecenter/en/SSMKHH_10.0.0/com.ibm.etools.mft.doc/ak05270_.htm)
- **MySQL:** [Trim function](http://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim)
- **Oracle:** [ltrim](https://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2018.htm#OLADM594)
- **DB2:** [ltrim](https://www.ibm.com/support/knowledgecenter/en/SSEPEK_11.0.0/sqlref/src/tpc/db2z_bif_ltrim.html)
This PR is to implement the above enhancement. In the implementation, the design principle is to keep the changes to the minimum. Also, the exiting trim functions (which handles a special case, i.e., trimming space characters) are kept unchanged for performane reasons.
#### How was this patch tested?
The unit test cases are added in the following files:
- UTF8StringSuite.java
- StringExpressionsSuite.scala
- sql/SQLQuerySuite.scala
- StringFunctionsSuite.scala
Author: Kevin Yu <qyu@us.ibm.com>
Closes#12646 from kevinyu98/spark-14878.
## What changes were proposed in this pull request?
If there are two projects like as follows.
```
Project [a_with_metadata#27 AS b#26]
+- Project [a#0 AS a_with_metadata#27]
+- LocalRelation <empty>, [a#0, b#1]
```
Child Project has an output column with a metadata in it, and the parent Project has an alias that implicitly forwards the metadata. So this metadata is visible for higher operators. Upon applying CollapseProject optimizer rule, the metadata is not preserved.
```
Project [a#0 AS b#26]
+- LocalRelation <empty>, [a#0, b#1]
```
This is incorrect, as downstream operators that expect certain metadata (e.g. watermark in structured streaming) to identify certain fields will fail to do so. This PR fixes it by preserving the metadata of top-level aliases.
## How was this patch tested?
New unit test
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#19240 from tdas/SPARK-22018.
## What changes were proposed in this pull request?
In previous work SPARK-21513, we has allowed `MapType` and `ArrayType` of `MapType`s convert to a json string but only for Scala API. In this follow-up PR, we will make SparkSQL support it for PySpark and SparkR, too. We also fix some little bugs and comments of the previous work in this follow-up PR.
### For PySpark
```
>>> data = [(1, {"name": "Alice"})]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"name":"Alice")']
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
```
### For SparkR
```
# Converts a map into a JSON object
df2 <- sql("SELECT map('name', 'Bob')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
# Converts an array of maps into a JSON array
df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
```
## How was this patch tested?
Add unit test cases.
cc viirya HyukjinKwon
Author: goldmedal <liugs963@gmail.com>
Closes#19223 from goldmedal/SPARK-21513-fp-PySaprkAndSparkR.
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-21980
This PR fixes the issue in ResolveGroupingAnalytics rule, which indexes the column references in grouping functions without considering case sensitive configurations.
The problem can be reproduced by:
`val df = spark.createDataFrame(Seq((1, 1), (2, 1), (2, 2))).toDF("a", "b")
df.cube("a").agg(grouping("A")).show()`
## How was this patch tested?
unit tests
Author: donnyzone <wellfengzhu@gmail.com>
Closes#19202 from DonnyZone/ResolveGroupingAnalytics.
# What changes were proposed in this pull request?
UDF to_json only supports converting `StructType` or `ArrayType` of `StructType`s to a json output string now.
According to the discussion of JIRA SPARK-21513, I allow to `to_json` support converting `MapType` and `ArrayType` of `MapType`s to a json output string.
This PR is for SQL and Scala API only.
# How was this patch tested?
Adding unit test case.
cc viirya HyukjinKwon
Author: goldmedal <liugs963@gmail.com>
Author: Jia-Xuan Liu <liugs963@gmail.com>
Closes#18875 from goldmedal/SPARK-21513.
## What changes were proposed in this pull request?
Improve QueryPlanConstraints framework, make it robust and simple.
In https://github.com/apache/spark/pull/15319, constraints for expressions like `a = f(b, c)` is resolved.
However, for expressions like
```scala
a = f(b, c) && c = g(a, b)
```
The current QueryPlanConstraints framework will produce non-converging constraints.
Essentially, the problem is caused by having both the name and child of aliases in the same constraint set. We infer constraints, and push down constraints as predicates in filters, later on these predicates are propagated as constraints, etc..
Simply using the alias names only can resolve these problems. The size of constraints is reduced without losing any information. We can always get these inferred constraints on child of aliases when pushing down filters.
Also, the EqualNullSafe between name and child in propagating alias is meaningless
```scala
allConstraints += EqualNullSafe(e, a.toAttribute)
```
It just produces redundant constraints.
## How was this patch tested?
Unit test
Author: Wang Gengliang <ltnwgl@gmail.com>
Closes#19201 from gengliangwang/QueryPlanConstraints.
## What changes were proposed in this pull request?
Support DESC (EXTENDED | FORMATTED) ? TABLE COLUMN command.
Support DESC EXTENDED | FORMATTED TABLE COLUMN command to show column-level statistics.
Do NOT support describe nested columns.
## How was this patch tested?
Added test cases.
Author: Zhenhua Wang <wzh_zju@163.com>
Author: Zhenhua Wang <wangzhenhua@huawei.com>
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#16422 from wzhfy/descColumn.
## What changes were proposed in this pull request?
This PR implements the sql feature:
INSERT OVERWRITE [LOCAL] DIRECTORY directory1
[ROW FORMAT row_format] [STORED AS file_format]
SELECT ... FROM ...
## How was this patch tested?
Added new unittests and also pulled the code to fb-spark so that we could test writing to hdfs directory.
Author: Jane Wang <janewang@fb.com>
Closes#18975 from janewangfb/port_local_directory.
## What changes were proposed in this pull request?
`JacksonUtils.verifySchema` verifies if a data type can be converted to JSON. For `MapType`, it now verifies the key type. However, in `JacksonGenerator`, when converting a map to JSON, we only care about its values and create a writer for the values. The keys in a map are treated as strings by calling `toString` on the keys.
Thus, we should change `JacksonUtils.verifySchema` to verify the value type of `MapType`.
## How was this patch tested?
Added tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#19167 from viirya/test-jacksonutils.
## What changes were proposed in this pull request?
The condition in `Optimizer.isPlanIntegral` is wrong. We should always return `true` if not in test mode.
## How was this patch tested?
Manually test.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#19161 from viirya/SPARK-21726-followup.
## What changes were proposed in this pull request?
We have many optimization rules now in `Optimzer`. Right now we don't have any checks in the optimizer to check for the structural integrity of the plan (e.g. resolved). When debugging, it is difficult to identify which rules return invalid plans.
It would be great if in test mode, we can check whether a plan is still resolved after the execution of each rule, so we can catch rules that return invalid plans.
## How was this patch tested?
Added tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18956 from viirya/SPARK-21726.
## What changes were proposed in this pull request?
Since [SPARK-15639](https://github.com/apache/spark/pull/13701), `spark.sql.parquet.cacheMetadata` and `PARQUET_CACHE_METADATA` is not used. This PR removes from SQLConf and docs.
## How was this patch tested?
Pass the existing Jenkins.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#19129 from dongjoon-hyun/SPARK-13656.
## What changes were proposed in this pull request?
This is a follow-up of #19050 to deal with `ExistenceJoin` case.
## How was this patch tested?
Added test.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#19151 from viirya/SPARK-21835-followup.
## What changes were proposed in this pull request?
Add an assert in logical plan optimization that the isStreaming bit stays the same, and fix empty relation rules where that wasn't happening.
## How was this patch tested?
new and existing unit tests
Author: Jose Torres <joseph.torres@databricks.com>
Author: Jose Torres <joseph-torres@databricks.com>
Closes#19056 from joseph-torres/SPARK-21765-followup.
## What changes were proposed in this pull request?
Correlated predicate subqueries are rewritten into `Join` by the rule `RewritePredicateSubquery` during optimization.
It is possibly that the two sides of the `Join` have conflicting attributes. The query plans produced by `RewritePredicateSubquery` become unresolved and break structural integrity.
We should check if there are conflicting attributes in the `Join` and de-duplicate them by adding a `Project`.
## How was this patch tested?
Added tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#19050 from viirya/SPARK-21835.
## What changes were proposed in this pull request?
For the given example below, the predicate added by `InferFiltersFromConstraints` is folded by `ConstantPropagation` later, this leads to unconverged optimize iteration:
```
Seq((1, 1)).toDF("col1", "col2").createOrReplaceTempView("t1")
Seq(1, 2).toDF("col").createOrReplaceTempView("t2")
sql("SELECT * FROM t1, t2 WHERE t1.col1 = 1 AND 1 = t1.col2 AND t1.col1 = t2.col AND t1.col2 = t2.col")
```
We can fix this by adjusting the indent of the optimize rules.
## How was this patch tested?
Add test case that would have failed in `SQLQuerySuite`.
Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Closes#19099 from jiangxb1987/unconverge-optimization.
## What changes were proposed in this pull request?
We should make codegen fallback of expressions configurable. So far, it is always on. We might hide it when our codegen have compilation bugs. Thus, we should also disable the codegen fallback when running test cases.
## How was this patch tested?
Added test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#19119 from gatorsmile/fallbackCodegen.
## What changes were proposed in this pull request?
SQL predicates don't have complete expression description. This patch goes to complement the description by adding arguments, examples.
This change also adds related test cases for the SQL predicate expressions.
## How was this patch tested?
Existing tests. And added predicate test.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18869 from viirya/SPARK-21654.
## What changes were proposed in this pull request?
Add `TBLPROPERTIES` to the DDL statement `CREATE TABLE USING`.
After this change, the DDL becomes
```
CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
USING table_provider
[OPTIONS table_property_list]
[PARTITIONED BY (col_name, col_name, ...)]
[CLUSTERED BY (col_name, col_name, ...)
[SORTED BY (col_name [ASC|DESC], ...)]
INTO num_buckets BUCKETS
]
[LOCATION path]
[COMMENT table_comment]
[TBLPROPERTIES (property_name=property_value, ...)]
[[AS] select_statement];
```
## How was this patch tested?
Add a few tests
Author: gatorsmile <gatorsmile@gmail.com>
Closes#19100 from gatorsmile/addTablePropsToCreateTableUsing.
…build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure
## What changes were proposed in this pull request?
This change adds back the infrastructure for a Scala 2.12 build, but does not enable it in the release or Python test scripts.
In order to make that meaningful, it also resolves compile errors that the code hits in 2.12 only, in a way that still works with 2.11.
It also updates dependencies to the earliest minor release of dependencies whose current version does not yet support Scala 2.12. This is in a sense covered by other JIRAs under the main umbrella, but implemented here. The versions below still work with 2.11, and are the _latest_ maintenance release in the _earliest_ viable minor release.
- Scalatest 2.x -> 3.0.3
- Chill 0.8.0 -> 0.8.4
- Clapper 1.0.x -> 1.1.2
- json4s 3.2.x -> 3.4.2
- Jackson 2.6.x -> 2.7.9 (required by json4s)
This change does _not_ fully enable a Scala 2.12 build:
- It will also require dropping support for Kafka before 0.10. Easy enough, just didn't do it yet here
- It will require recreating `SparkILoop` and `Main` for REPL 2.12, which is SPARK-14650. Possible to do here too.
What it does do is make changes that resolve much of the remaining gap without affecting the current 2.11 build.
## How was this patch tested?
Existing tests and build. Manually tested with `./dev/change-scala-version.sh 2.12` to verify it compiles, modulo the exceptions above.
Author: Sean Owen <sowen@cloudera.com>
Closes#18645 from srowen/SPARK-14280.
## What changes were proposed in this pull request?
Allows `BinaryComparison` operators to work on any data type that actually supports ordering as verified by `TypeUtils.checkForOrderingExpr` instead of relying on the incomplete list `TypeCollection.Ordered` (which is removed by this PR).
## How was this patch tested?
Updated unit tests to cover structs and arrays.
Author: Andrew Ray <ray.andrew@gmail.com>
Closes#18818 from aray/SPARK-21110.
## What changes were proposed in this pull request?
Also remove useless function `partitionByDeterministic` after the changes of https://github.com/apache/spark/pull/14687
## How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#19097 from gatorsmile/followupSPARK-17107.
## What changes were proposed in this pull request?
This PR aims to support `spark.sql.orc.compression.codec` like Parquet's `spark.sql.parquet.compression.codec`. Users can use SQLConf to control ORC compression, too.
## How was this patch tested?
Pass the Jenkins with new and updated test cases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#19055 from dongjoon-hyun/SPARK-21839.
## What changes were proposed in this pull request?
We should make codegen fallback of expressions configurable. So far, it is always on. We might hide it when our codegen have compilation bugs. Thus, we should also disable the codegen fallback when running test cases.
## How was this patch tested?
Added test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#19062 from gatorsmile/fallbackCodegen.
## What changes were proposed in this pull request?
This is a follow-up for https://github.com/apache/spark/pull/18488, to simplify the code.
The major change is, we should map java enum to string type, instead of a struct type with a single string field.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#19066 from cloud-fan/fix.
## What changes were proposed in this pull request?
Add trait UserDefinedExpression to identify user-defined functions.
UDF can be expensive. In optimizer we may need to avoid executing UDF multiple times.
E.g.
```scala
table.select(UDF as 'a).select('a, ('a + 1) as 'b)
```
If UDF is expensive in this case, optimizer should not collapse the project to
```scala
table.select(UDF as 'a, (UDF+1) as 'b)
```
Currently UDF classes like PythonUDF, HiveGenericUDF are not defined in catalyst.
This PR is to add a new trait to make it easier to identify user-defined functions.
## How was this patch tested?
Unit test
Author: Wang Gengliang <ltnwgl@gmail.com>
Closes#19064 from gengliangwang/UDFType.
## What changes were proposed in this pull request?
This patch adds allowUnquotedControlChars option in JSON data source to allow JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters)
## How was this patch tested?
Add new test cases
Author: vinodkc <vinod.kc.in@gmail.com>
Closes#19008 from vinodkc/br_fix_SPARK-21756.
## What changes were proposed in this pull request?
Fixed NPE when creating encoder for enum.
When you try to create an encoder for Enum type (or bean with enum property) via Encoders.bean(...), it fails with NullPointerException at TypeToken:495.
I did a little research and it turns out, that in JavaTypeInference following code
```
def getJavaBeanReadableProperties(beanClass: Class[_]): Array[PropertyDescriptor] = {
val beanInfo = Introspector.getBeanInfo(beanClass)
beanInfo.getPropertyDescriptors.filterNot(_.getName == "class")
.filter(_.getReadMethod != null)
}
```
filters out properties named "class", because we wouldn't want to serialize that. But enum types have another property of type Class named "declaringClass", which we are trying to inspect recursively. Eventually we try to inspect ClassLoader class, which has property "defaultAssertionStatus" with no read method, which leads to NPE at TypeToken:495.
I added property name "declaringClass" to filtering to resolve this.
## How was this patch tested?
Unit test in JavaDatasetSuite which creates an encoder for enum
Author: mike <mike0sv@gmail.com>
Author: Mikhail Sveshnikov <mike0sv@gmail.com>
Closes#18488 from mike0sv/enum-support.
## What changes were proposed in this pull request?
This PR bumps the ANTLR version to 4.7, and fixes a number of small parser related issues uncovered by the bump.
The main reason for upgrading is that in some cases the current version of ANTLR (4.5) can exhibit exponential slowdowns if it needs to parse boolean predicates. For example the following query will take forever to parse:
```sql
SELECT *
FROM RANGE(1000)
WHERE
TRUE
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
AND NOT upper(DESCRIPTION) LIKE '%FOO%'
```
This is caused by a know bug in ANTLR (https://github.com/antlr/antlr4/issues/994), which was fixed in version 4.6.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#19042 from hvanhovell/SPARK-21830.
## What changes were proposed in this pull request?
With the check for structural integrity proposed in SPARK-21726, it is found that the optimization rule `PullupCorrelatedPredicates` can produce unresolved plans.
For a correlated IN query looks like:
SELECT t1.a FROM t1
WHERE
t1.a IN (SELECT t2.c
FROM t2
WHERE t1.b < t2.d);
The query plan might look like:
Project [a#0]
+- Filter a#0 IN (list#4 [b#1])
: +- Project [c#2]
: +- Filter (outer(b#1) < d#3)
: +- LocalRelation <empty>, [c#2, d#3]
+- LocalRelation <empty>, [a#0, b#1]
After `PullupCorrelatedPredicates`, it produces query plan like:
'Project [a#0]
+- 'Filter a#0 IN (list#4 [(b#1 < d#3)])
: +- Project [c#2, d#3]
: +- LocalRelation <empty>, [c#2, d#3]
+- LocalRelation <empty>, [a#0, b#1]
Because the correlated predicate involves another attribute `d#3` in subquery, it has been pulled out and added into the `Project` on the top of the subquery.
When `list` in `In` contains just one `ListQuery`, `In.checkInputDataTypes` checks if the size of `value` expressions matches the output size of subquery. In the above example, there is only `value` expression and the subquery output has two attributes `c#2, d#3`, so it fails the check and `In.resolved` returns `false`.
We should not let `In.checkInputDataTypes` wrongly report unresolved plans to fail the structural integrity check.
## How was this patch tested?
Added test.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18968 from viirya/SPARK-21759.
## What changes were proposed in this pull request?
This is a refactoring of `ColumnVector` hierarchy and related classes.
1. make `ColumnVector` read-only
2. introduce `WritableColumnVector` with write interface
3. remove `ReadOnlyColumnVector`
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#18958 from ueshin/issues/SPARK-21745.
## What changes were proposed in this pull request?
When json_tuple in extracting values from JSON it returns null values within repeated columns except the first one as below:
``` scala
scala> spark.sql("""SELECT json_tuple('{"a":1, "b":2}', 'a', 'b', 'a')""").show()
+---+---+----+
| c0| c1| c2|
+---+---+----+
| 1| 2|null|
+---+---+----+
```
I think this should be consistent with Hive's implementation:
```
hive> SELECT json_tuple('{"a": 1, "b": 2}', 'a', 'a');
...
1 1
```
In this PR, we located all the matched indices in `fieldNames` instead of returning the first matched index, i.e., indexOf.
## How was this patch tested?
Added test in JsonExpressionsSuite.
Author: Jen-Ming Chung <jenmingisme@gmail.com>
Closes#19017 from jmchung/SPARK-21804.
## What changes were proposed in this pull request?
The getAliasedConstraints fuction in LogicalPlan.scala will clone the expression set when an element added,
and it will take a long time. This PR add a function to add multiple elements at once to reduce the clone time.
Before modified, the cost of getAliasedConstraints is:
100 expressions: 41 seconds
150 expressions: 466 seconds
After modified, the cost of getAliasedConstraints is:
100 expressions: 1.8 seconds
150 expressions: 6.5 seconds
The test is like this:
test("getAliasedConstraints") {
val expressionNum = 150
val aggExpression = (1 to expressionNum).map(i => Alias(Count(Literal(1)), s"cnt$i")())
val aggPlan = Aggregate(Nil, aggExpression, LocalRelation())
val beginTime = System.currentTimeMillis()
val expressions = aggPlan.validConstraints
println(s"validConstraints cost: ${System.currentTimeMillis() - beginTime}ms")
// The size of Aliased expression is n * (n - 1) / 2 + n
assert( expressions.size === expressionNum * (expressionNum - 1) / 2 + expressionNum)
}
(Please fill in changes proposed in this fix)
## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Run new added test.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: 10129659 <chen.yanshan@zte.com.cn>
Closes#19022 from eatoncys/getAliasedConstraints.
## What changes were proposed in this pull request?
This pr changed the default value of `maxLinesPerFunction` into `4000`. In #18810, we had this new option to disable code generation for too long functions and I found this option only affected `Q17` and `Q66` in TPC-DS. But, `Q66` had some performance regression:
```
Q17 w/o #18810, 3224ms --> q17 w/#18810, 2627ms (improvement)
Q66 w/o #18810, 1712ms --> q66 w/#18810, 3032ms (regression)
```
To keep the previous performance in TPC-DS, we better set higher value at `maxLinesPerFunction` by default.
## How was this patch tested?
Existing tests.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#19021 from maropu/SPARK-21603-FOLLOWUP-1.
## What changes were proposed in this pull request?
All streaming logical plans will now have isStreaming set. This involved adding isStreaming as a case class arg in a few cases, since a node might be logically streaming depending on where it came from.
## How was this patch tested?
Existing unit tests - no functional change is intended in this PR.
Author: Jose Torres <joseph-torres@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#18973 from joseph-torres/SPARK-21765.
## What changes were proposed in this pull request?
This PR is to enable users to create persistent Scala UDAF (that extends UserDefinedAggregateFunction).
```SQL
CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg'
```
Before this PR, Spark UDAF only can be registered through the API `spark.udf.register(...)`
## How was this patch tested?
Added test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18700 from gatorsmile/javaUDFinScala.
## What changes were proposed in this pull request?
This is a follow-up of https://github.com/apache/spark/pull/18955 , to fix a bug that we break whole stage codegen for `Limit`.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18993 from cloud-fan/bug.
## What changes were proposed in this pull request?
Added support for ANALYZE TABLE [db_name].tablename PARTITION (partcol1[=val1], partcol2[=val2], ...) COMPUTE STATISTICS [NOSCAN] SQL command to calculate total number of rows and size in bytes for a subset of partitions. Calculated statistics are stored in Hive Metastore as user-defined properties attached to partition objects. Property names are the same as the ones used to store table-level statistics: spark.sql.statistics.totalSize and spark.sql.statistics.numRows.
When partition specification contains all partition columns with values, the command collects statistics for a single partition that matches the specification. When some partition columns are missing or listed without their values, the command collects statistics for all partitions which match a subset of partition column values specified.
For example, table t has 4 partitions with the following specs:
* Partition1: (ds='2008-04-08', hr=11)
* Partition2: (ds='2008-04-08', hr=12)
* Partition3: (ds='2008-04-09', hr=11)
* Partition4: (ds='2008-04-09', hr=12)
'ANALYZE TABLE t PARTITION (ds='2008-04-09', hr=11)' command will collect statistics only for partition 3.
'ANALYZE TABLE t PARTITION (ds='2008-04-09')' command will collect statistics for partitions 3 and 4.
'ANALYZE TABLE t PARTITION (ds, hr)' command will collect statistics for all four partitions.
When the optional parameter NOSCAN is specified, the command doesn't count number of rows and only gathers size in bytes.
The statistics gathered by ANALYZE TABLE command can be fetched using DESC EXTENDED [db_name.]tablename PARTITION command.
## How was this patch tested?
Added tests.
Author: Masha Basmanova <mbasmanova@fb.com>
Closes#18421 from mbasmanova/mbasmanova-analyze-partition.
## What changes were proposed in this pull request?
``` scala
scala> Seq(("""{"Hyukjin": 224, "John": 1225}""")).toDS.selectExpr("json_tuple(value, trim(null))").show()
...
java.lang.NullPointerException
at ...
```
Currently the `null` field name will throw NullPointException. As a given field name null can't be matched with any field names in json, we just output null as its column value. This PR achieves it by returning a very unlikely column name `__NullFieldName` in evaluation of the field names.
## How was this patch tested?
Added unit test.
Author: Jen-Ming Chung <jenmingisme@gmail.com>
Closes#18930 from jmchung/SPARK-21677.
## What changes were proposed in this pull request?
This pr sorted output attributes on their name and exprId in `AttributeSet.toSeq` to make the order consistent. If the order is different, spark possibly generates different code and then misses cache in `CodeGenerator`, e.g., `GenerateColumnAccessor` generates code depending on an input attribute order.
## How was this patch tested?
Added tests in `AttributeSetSuite` and manually checked if the cache worked well in the given query of the JIRA.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18959 from maropu/SPARK-18394.
## What changes were proposed in this pull request?
Close the whole stage codegen when the function lines is longer than the maxlines which will be setted by
spark.sql.codegen.MaxFunctionLength parameter, because when the function is too long , it will not get the JIT optimizing.
A benchmark test result is 10x slower when the generated function is too long :
ignore("max function length of wholestagecodegen") {
val N = 20 << 15
val benchmark = new Benchmark("max function length of wholestagecodegen", N)
def f(): Unit = sparkSession.range(N)
.selectExpr(
"id",
"(id & 1023) as k1",
"cast(id & 1023 as double) as k2",
"cast(id & 1023 as int) as k3",
"case when id > 100 and id <= 200 then 1 else 0 end as v1",
"case when id > 200 and id <= 300 then 1 else 0 end as v2",
"case when id > 300 and id <= 400 then 1 else 0 end as v3",
"case when id > 400 and id <= 500 then 1 else 0 end as v4",
"case when id > 500 and id <= 600 then 1 else 0 end as v5",
"case when id > 600 and id <= 700 then 1 else 0 end as v6",
"case when id > 700 and id <= 800 then 1 else 0 end as v7",
"case when id > 800 and id <= 900 then 1 else 0 end as v8",
"case when id > 900 and id <= 1000 then 1 else 0 end as v9",
"case when id > 1000 and id <= 1100 then 1 else 0 end as v10",
"case when id > 1100 and id <= 1200 then 1 else 0 end as v11",
"case when id > 1200 and id <= 1300 then 1 else 0 end as v12",
"case when id > 1300 and id <= 1400 then 1 else 0 end as v13",
"case when id > 1400 and id <= 1500 then 1 else 0 end as v14",
"case when id > 1500 and id <= 1600 then 1 else 0 end as v15",
"case when id > 1600 and id <= 1700 then 1 else 0 end as v16",
"case when id > 1700 and id <= 1800 then 1 else 0 end as v17",
"case when id > 1800 and id <= 1900 then 1 else 0 end as v18")
.groupBy("k1", "k2", "k3")
.sum()
.collect()
benchmark.addCase(s"codegen = F") { iter =>
sparkSession.conf.set("spark.sql.codegen.wholeStage", "false")
f()
}
benchmark.addCase(s"codegen = T") { iter =>
sparkSession.conf.set("spark.sql.codegen.wholeStage", "true")
sparkSession.conf.set("spark.sql.codegen.MaxFunctionLength", "10000")
f()
}
benchmark.run()
/*
Java HotSpot(TM) 64-Bit Server VM 1.8.0_111-b14 on Windows 7 6.1
Intel64 Family 6 Model 58 Stepping 9, GenuineIntel
max function length of wholestagecodegen: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
codegen = F 443 / 507 1.5 676.0 1.0X
codegen = T 3279 / 3283 0.2 5002.6 0.1X
*/
}
## How was this patch tested?
Run the unit test
Author: 10129659 <chen.yanshan@zte.com.cn>
Closes#18810 from eatoncys/codegen.
## What changes were proposed in this pull request?
This patch adds the DataFrames API to the multivariate summarizer (mean, variance, etc.). In addition to all the features of MultivariateOnlineSummarizer, it also allows the user to select a subset of the metrics.
## How was this patch tested?
Testcases added.
## Performance
Resolve several performance issues in #17419, further optimization pending on SQL team's work. One of the SQL layer performance issue related to these feature has been resolved in #18712, thanks liancheng and cloud-fan
### Performance data
(test on my laptop, use 2 partitions. tries out = 20, warm up = 10)
The unit of test results is records/milliseconds (higher is better)
Vector size/records number | 1/10000000 | 10/1000000 | 100/1000000 | 1000/100000 | 10000/10000
----|------|----|---|----|----
Dataframe | 15149 | 7441 | 2118 | 224 | 21
RDD from Dataframe | 4992 | 4440 | 2328 | 320 | 33
raw RDD | 53931 | 20683 | 3966 | 528 | 53
Author: WeichenXu <WeichenXu123@outlook.com>
Closes#18798 from WeichenXu123/SPARK-19634-dataframe-summarizer.
This version fixes a few issues in the import order checker; it provides
better error messages, and detects more improper ordering (thus the need
to change a lot of files in this patch). The main fix is that it correctly
complains about the order of packages vs. classes.
As part of the above, I moved some "SparkSession" import in ML examples
inside the "$example on$" blocks; that didn't seem consistent across
different source files to start with, and avoids having to add more on/off blocks
around specific imports.
The new scalastyle also seems to have a better header detector, so a few
license headers had to be updated to match the expected indentation.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#18943 from vanzin/SPARK-21731.
## What changes were proposed in this pull request?
This is a follow-up of https://github.com/apache/spark/pull/15900 , to fix one more bug:
When table schema is empty and need to be inferred at runtime, we should not resolve parent plans before the schema has been inferred, or the parent plans will be resolved against an empty schema and may get wrong result for something like `select *`
The fix logic is: introduce `UnresolvedCatalogRelation` as a placeholder. Then we replace it with `LogicalRelation` or `HiveTableRelation` during analysis, so that it's guaranteed that we won't resolve parent plans until the schema has been inferred.
## How was this patch tested?
regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18907 from cloud-fan/bug.
## What changes were proposed in this pull request?
This PR adds `since` annotation in documentation so that this can be rendered as below:
<img width="290" alt="2017-08-14 6 54 26" src="https://user-images.githubusercontent.com/6477701/29267050-034c1f64-8122-11e7-862b-7dfc38e292bf.png">
## How was this patch tested?
Manually checked the documentation by `cd sql && ./create-docs.sh`.
Also, Jenkins tests are required.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18939 from HyukjinKwon/add-sinces-date-functions.
## What changes were proposed in this pull request?
While discovering optimization rules and their test coverage, I did not find any tests for `CheckCartesianProducts` in the Catalyst folder. So, I decided to create a new test suite. Once I finished, I found a test in `JoinSuite` for this functionality so feel free to discard this change if it does not make much sense. The proposed test suite covers a few additional use cases.
Author: aokolnychyi <anton.okolnychyi@sap.com>
Closes#18909 from aokolnychyi/check-cartesian-join-tests.
## What changes were proposed in this pull request?
[SPARK-21595](https://issues.apache.org/jira/browse/SPARK-21595) reported that there is excessive spilling to disk due to default spill threshold for `ExternalAppendOnlyUnsafeRowArray` being quite small for WINDOW operator. Old behaviour of WINDOW operator (pre https://github.com/apache/spark/pull/16909) would hold data in an array for first 4096 records post which it would switch to `UnsafeExternalSorter` and start spilling to disk after reaching `spark.shuffle.spill.numElementsForceSpillThreshold` (or earlier if there was paucity of memory due to excessive consumers).
Currently the (switch from in-memory to `UnsafeExternalSorter`) and (`UnsafeExternalSorter` spilling to disk) for `ExternalAppendOnlyUnsafeRowArray` is controlled by a single threshold. This PR aims to separate that to have more granular control.
## How was this patch tested?
Added unit tests
Author: Tejas Patil <tejasp@fb.com>
Closes#18843 from tejasapatil/SPARK-21595.
## What changes were proposed in this pull request?
This patch removes the unused SessionCatalog.getTableMetadataOption and ExternalCatalog. getTableOption.
## How was this patch tested?
Removed the test case.
Author: Reynold Xin <rxin@databricks.com>
Closes#18912 from rxin/remove-getTableOption.
## What changes were proposed in this pull request?
Push filter predicates through EventTimeWatermark if they're deterministic and do not reference the watermarked attribute. (This is similar but not identical to the logic for pushing through UnaryNode.)
## How was this patch tested?
unit tests
Author: Jose Torres <joseph-torres@databricks.com>
Closes#18790 from joseph-torres/SPARK-21587.
## What changes were proposed in this pull request?
This PR is to add the spark version info in the table metadata. When creating the table, this value is assigned. It can help users find which version of Spark was used to create the table.
## How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18709 from gatorsmile/addVersion.
## What changes were proposed in this pull request?
Window rangeBetween() API should allow literal boundary, that means, the window range frame can calculate frame of double/date/timestamp.
Example of the use case can be:
```
SELECT
val_timestamp,
cate,
avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING)
FROM testData
```
This PR refactors the Window `rangeBetween` and `rowsBetween` API, while the legacy user code should still be valid.
## How was this patch tested?
Add new test cases both in `DataFrameWindowFunctionsSuite` and in `window.sql`.
Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Closes#18814 from jiangxb1987/literal-boundary.
## What changes were proposed in this pull request?
If we create a type alias for a type workable with Dataset, the type alias doesn't work with Dataset.
A reproducible case looks like:
object C {
type TwoInt = (Int, Int)
def tupleTypeAlias: TwoInt = (1, 1)
}
Seq(1).toDS().map(_ => ("", C.tupleTypeAlias))
It throws an exception like:
type T1 is not a class
scala.ScalaReflectionException: type T1 is not a class
at scala.reflect.api.Symbols$SymbolApi$class.asClass(Symbols.scala:275)
...
This patch accesses the dealias of type in many places in `ScalaReflection` to fix it.
## How was this patch tested?
Added test case.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18813 from viirya/SPARK-21567.
I have discovered that "full_outer" name option is working in Spark 2.0, but it is not printed in exception. Please verify.
## What changes were proposed in this pull request?
(Please fill in changes proposed in this fix)
## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: BartekH <bartekhamielec@gmail.com>
Closes#17985 from BartekH/patch-1.
## What changes were proposed in this pull request?
This pr (follow-up of #18772) used `UnresolvedSubqueryColumnAliases` for `visitTableName` in `AstBuilder`, which is a new unresolved `LogicalPlan` implemented in #18185.
## How was this patch tested?
Existing tests
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18857 from maropu/SPARK-20963-FOLLOWUP.
## What changes were proposed in this pull request?
In SQLContext.get(key,null) for a key that is not defined in the conf, and doesn't have a default value defined, throws a NPE. Int happens only when conf has a value converter
Added null check on defaultValue inside SQLConf.getConfString to avoid calling entry.valueConverter(defaultValue)
## How was this patch tested?
Added unit test
Author: vinodkc <vinod.kc.in@gmail.com>
Closes#18852 from vinodkc/br_Fix_SPARK-21588.
## What changes were proposed in this pull request?
This pr added parsing rules to support column aliases for join relations in FROM clause.
This pr is a sub-task of #18079.
## How was this patch tested?
Added tests in `AnalysisSuite`, `PlanParserSuite,` and `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18772 from maropu/SPARK-20963-2.
## What changes were proposed in this pull request?
This PR proposes to separate `extended` into `examples` and `arguments` internally so that both can be separately documented and add `since` and `note` for additional information.
For `since`, it looks users sometimes get confused by, up to my knowledge, missing version information. For example, see https://www.mail-archive.com/userspark.apache.org/msg64798.html
For few good examples to check the built documentation, please see both:
`from_json` - https://spark-test.github.io/sparksqldoc/#from_json
`like` - https://spark-test.github.io/sparksqldoc/#like
For `DESCRIBE FUNCTION`, `note` and `since` are added as below:
```
> DESCRIBE FUNCTION EXTENDED rlike;
...
Extended Usage:
Arguments:
...
Examples:
...
Note:
Use LIKE to match with simple string pattern
```
```
> DESCRIBE FUNCTION EXTENDED to_json;
...
Examples:
...
Since: 2.2.0
```
For the complete documentation, see https://spark-test.github.io/sparksqldoc/
## How was this patch tested?
Manual tests and existing tests. Please see https://spark-test.github.io/sparksqldoc
Jenkins tests are needed to double check
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18749 from HyukjinKwon/followup-sql-doc-gen.
## What changes were proposed in this pull request?
create temporary view data as select * from values
(1, 1),
(1, 2),
(2, 1),
(2, 2),
(3, 1),
(3, 2)
as data(a, b);
`select 3, 4, sum(b) from data group by 1, 2;`
`select 3 as c, 4 as d, sum(b) from data group by c, d;`
When running these two cases, the following exception occurred:
`Error in query: GROUP BY position 4 is not in select list (valid range is [1, 3]); line 1 pos 10`
The cause of this failure:
If an aggregateExpression is integer, after replaced with this aggregateExpression, the
groupExpression still considered as an ordinal.
The solution:
This bug is due to re-entrance of an analyzed plan. We can solve it by using `resolveOperators` in `SubstituteUnresolvedOrdinals`.
## How was this patch tested?
Added unit test case
Author: liuxian <liu.xian3@zte.com.cn>
Closes#18779 from 10110346/groupby.
## What changes were proposed in this pull request?
OneRowRelation is the only plan that is a case object, which causes some issues with makeCopy using a 0-arg constructor. This patch changes it from a case object to a case class.
This blocks SPARK-21619.
## How was this patch tested?
Should be covered by existing test cases.
Author: Reynold Xin <rxin@databricks.com>
Closes#18839 from rxin/SPARK-21634.
## What changes were proposed in this pull request?
Hive `pmod(3.13, 0)`:
```:sql
hive> select pmod(3.13, 0);
OK
NULL
Time taken: 2.514 seconds, Fetched: 1 row(s)
hive>
```
Spark `mod(3.13, 0)`:
```:sql
spark-sql> select mod(3.13, 0);
NULL
spark-sql>
```
But the Spark `pmod(3.13, 0)`:
```:sql
spark-sql> select pmod(3.13, 0);
17/06/25 09:35:58 ERROR SparkSQLDriver: Failed in [select pmod(3.13, 0)]
java.lang.NullPointerException
at org.apache.spark.sql.catalyst.expressions.Pmod.pmod(arithmetic.scala:504)
at org.apache.spark.sql.catalyst.expressions.Pmod.nullSafeEval(arithmetic.scala:432)
at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:419)
at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:323)
...
```
This PR make `pmod(number, 0)` to null.
## How was this patch tested?
unit tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18413 from wangyum/SPARK-21205.
## What changes were proposed in this pull request?
Currently, StructType.merge() only reports data types of conflicting fields when merging two incompatible schemas. It would be nice to also report the field names for easier debugging.
## How was this patch tested?
Unit test in DataTypeSuite.
Print exception message when conflict is triggered.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes#16365 from bravo-zhang/spark-18950.
## What changes were proposed in this pull request?
This pr added parsing rules to support subquery column aliases in FROM clause.
This pr is a sub-task of #18079.
## How was this patch tested?
Added tests in `PlanParserSuite` and `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18185 from maropu/SPARK-20962.
## What changes were proposed in this pull request?
Long values can be passed to `rangeBetween` as range frame boundaries, but we silently convert it to Int values, this can cause wrong results and we should fix this.
Further more, we should accept any legal literal values as range frame boundaries. In this PR, we make it possible for Long values, and make accepting other DataTypes really easy to add.
This PR is mostly based on Herman's previous amazing work: 596f53c339
After this been merged, we can close#16818 .
## How was this patch tested?
Add new tests in `DataFrameWindowFunctionsSuite` and `TypeCoercionSuite`.
Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Closes#18540 from jiangxb1987/rangeFrame.
## What changes were proposed in this pull request?
When there are aliases (these aliases were added for nested fields) as parameters in `RuntimeReplaceable`, as they are not in the children expression, those aliases can't be cleaned up in analyzer rule `CleanupAliases`.
An expression `nvl(foo.foo1, "value")` can be resolved to two semantically different expressions in a group by query because they contain different aliases.
Because those aliases are not children of `RuntimeReplaceable` which is an `UnaryExpression`. So we can't trim the aliases out by simple transforming the expressions in `CleanupAliases`.
If we want to replace the non-children aliases in `RuntimeReplaceable`, we need to add more codes to `RuntimeReplaceable` and modify all expressions of `RuntimeReplaceable`. It makes the interface ugly IMO.
Consider those aliases will be replaced later at optimization and so they're no harm, this patch chooses to simply override `canonicalized` of `RuntimeReplaceable`.
One concern is about `CleanupAliases`. Because it actually cannot clean up ALL aliases inside a plan. To make caller of this rule notice that, this patch adds a comment to `CleanupAliases`.
## How was this patch tested?
Added test.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18761 from viirya/SPARK-21555.
## What changes were proposed in this pull request?
`UnsafeExternalSorter.recordComparator` can be either `KVComparator` or `RowComparator`, and both of them will keep the reference to the input rows they compared last time.
After sorting, we return the sorted iterator to upstream operators. However, the upstream operators may take a while to consume up the sorted iterator, and `UnsafeExternalSorter` is registered to `TaskContext` at [here](https://github.com/apache/spark/blob/v2.2.0/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java#L159-L161), which means we will keep the `UnsafeExternalSorter` instance and keep the last compared input rows in memory until the sorted iterator is consumed up.
Things get worse if we sort within partitions of a dataset and coalesce all partitions into one, as we will keep a lot of input rows in memory and the time to consume up all the sorted iterators is long.
This PR takes over https://github.com/apache/spark/pull/18543 , the idea is that, we do not keep the record comparator instance in `UnsafeExternalSorter`, but a generator of record comparator.
close#18543
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18679 from cloud-fan/memory-leak.
## What changes were proposed in this pull request?
This PR ensures that `Unsafe.sizeInBytes` must be a multiple of 8. It it is not satisfied. `Unsafe.hashCode` causes the assertion violation.
## How was this patch tested?
Will add test cases
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#18503 from kiszk/SPARK-21271.
### What changes were proposed in this pull request?
Like [Hive UDFType](https://hive.apache.org/javadocs/r2.0.1/api/org/apache/hadoop/hive/ql/udf/UDFType.html), we should allow users to add the extra flags for ScalaUDF and JavaUDF too. _stateful_/_impliesOrder_ are not applicable to our Scala UDF. Thus, we only add the following two flags.
- deterministic: Certain optimizations should not be applied if UDF is not deterministic. Deterministic UDF returns same result each time it is invoked with a particular input. This determinism just needs to hold within the context of a query.
When the deterministic flag is not correctly set, the results could be wrong.
For ScalaUDF in Dataset APIs, users can call the following extra APIs for `UserDefinedFunction` to make the corresponding changes.
- `nonDeterministic`: Updates UserDefinedFunction to non-deterministic.
Also fixed the Java UDF name loss issue.
Will submit a separate PR for `distinctLike` for UDAF
### How was this patch tested?
Added test cases for both ScalaUDF
Author: gatorsmile <gatorsmile@gmail.com>
Author: Wenchen Fan <cloud0fan@gmail.com>
Closes#17848 from gatorsmile/udfRegister.
## What changes were proposed in this pull request?
When the code that is generated is greater than 64k, then Janino compile will fail and CodeGenerator.scala will log the entire code at Error level.
SPARK-20871 suggests only logging the code at Debug level.
Since, the code is already logged at debug level, this Pull Request proposes not including the formatted code in the Error logging and exception message at all.
When an exception occurs, the code will be logged at Info level but truncated if it is more than 1000 lines long.
## How was this patch tested?
Existing tests were run.
An extra test test case was added to CodeFormatterSuite to test the new maxLines parameter,
Author: pj.fanning <pj.fanning@workday.com>
Closes#18658 from pjfanning/SPARK-20871.
## What changes were proposed in this pull request?
DirectParquetOutputCommitter was removed from Spark as it was deemed unsafe to use. We however still have some code to generate warning. This patch removes those code as well.
This is kind of a follow-up of https://github.com/apache/spark/pull/16796
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18689 from cloud-fan/minor.
## What changes were proposed in this pull request?
When we list partitions from hive metastore with a partial partition spec, we are expecting exact matching according to the partition values. However, hive treats dot specially and match any single character for dot. We should do an extra filter to drop unexpected partitions.
## How was this patch tested?
new regression test.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18671 from cloud-fan/hive.
## What changes were proposed in this pull request?
Address scapegoat warnings for:
- BigDecimal double constructor
- Catching NPE
- Finalizer without super
- List.size is O(n)
- Prefer Seq.empty
- Prefer Set.empty
- reverse.map instead of reverseMap
- Type shadowing
- Unnecessary if condition.
- Use .log1p
- Var could be val
In some instances like Seq.empty, I avoided making the change even where valid in test code to keep the scope of the change smaller. Those issues are concerned with performance and it won't matter for tests.
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#18635 from srowen/Scapegoat1.
## What changes were proposed in this pull request?
This PR changes the direction of expression transformation in the DecimalPrecision rule. Previously, the expressions were transformed down, which led to incorrect result types when decimal expressions had other decimal expressions as their operands. The root cause of this issue was in visiting outer nodes before their children. Consider the example below:
```
val inputSchema = StructType(StructField("col", DecimalType(26, 6)) :: Nil)
val sc = spark.sparkContext
val rdd = sc.parallelize(1 to 2).map(_ => Row(BigDecimal(12)))
val df = spark.createDataFrame(rdd, inputSchema)
// Works correctly since no nested decimal expression is involved
// Expected result type: (26, 6) * (26, 6) = (38, 12)
df.select($"col" * $"col").explain(true)
df.select($"col" * $"col").printSchema()
// Gives a wrong result since there is a nested decimal expression that should be visited first
// Expected result type: ((26, 6) * (26, 6)) * (26, 6) = (38, 12) * (26, 6) = (38, 18)
df.select($"col" * $"col" * $"col").explain(true)
df.select($"col" * $"col" * $"col").printSchema()
```
The example above gives the following output:
```
// Correct result without sub-expressions
== Parsed Logical Plan ==
'Project [('col * 'col) AS (col * col)#4]
+- LogicalRDD [col#1]
== Analyzed Logical Plan ==
(col * col): decimal(38,12)
Project [CheckOverflow((promote_precision(cast(col#1 as decimal(26,6))) * promote_precision(cast(col#1 as decimal(26,6)))), DecimalType(38,12)) AS (col * col)#4]
+- LogicalRDD [col#1]
== Optimized Logical Plan ==
Project [CheckOverflow((col#1 * col#1), DecimalType(38,12)) AS (col * col)#4]
+- LogicalRDD [col#1]
== Physical Plan ==
*Project [CheckOverflow((col#1 * col#1), DecimalType(38,12)) AS (col * col)#4]
+- Scan ExistingRDD[col#1]
// Schema
root
|-- (col * col): decimal(38,12) (nullable = true)
// Incorrect result with sub-expressions
== Parsed Logical Plan ==
'Project [(('col * 'col) * 'col) AS ((col * col) * col)#11]
+- LogicalRDD [col#1]
== Analyzed Logical Plan ==
((col * col) * col): decimal(38,12)
Project [CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(col#1 as decimal(26,6))) * promote_precision(cast(col#1 as decimal(26,6)))), DecimalType(38,12)) as decimal(26,6))) * promote_precision(cast(col#1 as decimal(26,6)))), DecimalType(38,12)) AS ((col * col) * col)#11]
+- LogicalRDD [col#1]
== Optimized Logical Plan ==
Project [CheckOverflow((cast(CheckOverflow((col#1 * col#1), DecimalType(38,12)) as decimal(26,6)) * col#1), DecimalType(38,12)) AS ((col * col) * col)#11]
+- LogicalRDD [col#1]
== Physical Plan ==
*Project [CheckOverflow((cast(CheckOverflow((col#1 * col#1), DecimalType(38,12)) as decimal(26,6)) * col#1), DecimalType(38,12)) AS ((col * col) * col)#11]
+- Scan ExistingRDD[col#1]
// Schema
root
|-- ((col * col) * col): decimal(38,12) (nullable = true)
```
## How was this patch tested?
This PR was tested with available unit tests. Moreover, there are tests to cover previously failing scenarios.
Author: aokolnychyi <anton.okolnychyi@sap.com>
Closes#18583 from aokolnychyi/spark-21332.
## What changes were proposed in this pull request?
Follow up to a few comments on https://github.com/apache/spark/pull/17150#issuecomment-315020196 that couldn't be addressed before it was merged.
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#18646 from srowen/SPARK-19810.2.
## What changes were proposed in this pull request?
This PR fixes a wrong comparison for `BinaryType`. This PR enables unsigned comparison and unsigned prefix generation for an array for `BinaryType`. Previous implementations uses signed operations.
## How was this patch tested?
Added a test suite in `OrderingSuite`.
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#18571 from kiszk/SPARK-21344.
## What changes were proposed in this pull request?
- Remove Scala 2.10 build profiles and support
- Replace some 2.10 support in scripts with commented placeholders for 2.12 later
- Remove deprecated API calls from 2.10 support
- Remove usages of deprecated context bounds where possible
- Remove Scala 2.10 workarounds like ScalaReflectionLock
- Other minor Scala warning fixes
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#17150 from srowen/SPARK-19810.
## What changes were proposed in this pull request?
Hive interprets regular expression, e.g., `(a)?+.+` in query specification. This PR enables spark to support this feature when hive.support.quoted.identifiers is set to true.
## How was this patch tested?
- Add unittests in SQLQuerySuite.scala
- Run spark-shell tested the original failed query:
scala> hc.sql("SELECT `(a|b)?+.+` from test1").collect.foreach(println)
Author: Jane Wang <janewang@fb.com>
Closes#18023 from janewangfb/support_select_regex.
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`. This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process. The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame. Data types except complex, date, timestamp, and decimal are currently supported, otherwise an `UnsupportedOperation` exception is thrown.
Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served. A package private class/object `ArrowConverters` that provide data type mappings and conversion routines. In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default).
## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types. The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data. This will ensure that the schema and data has been converted correctly.
Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow. A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.
Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>
Closes#18459 from BryanCutler/toPandas_with_arrow-SPARK-13534.
## What changes were proposed in this pull request?
This pr made it more consistent to handle column name duplication. In the current master, error handling is different when hitting column name duplication:
```
// json
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#12, a#13.;
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Duplicate column(s) : "a" found, cannot save to JSON format;
at org.apache.spark.sql.execution.datasources.json.JsonDataSource.checkConstraints(JsonDataSource.scala:81)
at org.apache.spark.sql.execution.datasources.json.JsonDataSource.inferSchema(JsonDataSource.scala:63)
at org.apache.spark.sql.execution.datasources.json.JsonFileFormat.inferSchema(JsonFileFormat.scala:57)
at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)
at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)
// csv
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#41, a#42.;
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)
// If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896)
scala> spark.read.format("csv").option("header", true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
| 1| 1|
+---+---+
// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#110, a#111.;
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```
When this patch applied, the results change to;
```
// json
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:156)
// csv
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
scala> spark.read.format("csv").option("header", true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
| 1| 1|
+---+---+
// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
```
## How was this patch tested?
Added tests in `DataFrameReaderWriterSuite` and `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17758 from maropu/SPARK-20460.
## What changes were proposed in this pull request?
These 3 methods have to be used together, so it makes more sense to merge them into one method and then the caller side only need to call one method.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18579 from cloud-fan/minor.
## What changes were proposed in this pull request?
Since we do not set active sessions when parsing the plan, we are unable to correctly use SQLConf.get to find the correct active session. Since https://github.com/apache/spark/pull/18531 breaks the build, I plan to revert it at first.
## How was this patch tested?
The existing test cases
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18568 from gatorsmile/revert18531.
## What changes were proposed in this pull request?
This pr modified code to use string types by default if `array` and `map` in functions have no argument. This behaviour is the same with Hive one;
```
hive> CREATE TEMPORARY TABLE t1 AS SELECT map();
hive> DESCRIBE t1;
_c0 map<string,string>
hive> CREATE TEMPORARY TABLE t2 AS SELECT array();
hive> DESCRIBE t2;
_c0 array<string>
```
## How was this patch tested?
Added tests in `DataFrameFunctionsSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18516 from maropu/SPARK-21281.
## What changes were proposed in this pull request?
un-aliased subquery is supported by Spark SQL for a long time. Its semantic was not well defined and had confusing behaviors, and it's not a standard SQL syntax, so we disallowed it in https://issues.apache.org/jira/browse/SPARK-20690 .
However, this is a breaking change, and we do have existing queries using un-aliased subquery. We should add the support back and fix its semantic.
This PR fixes the un-aliased subquery by assigning a default alias name.
After this PR, there is no syntax change from branch 2.2 to master, but we invalid a weird use case:
`SELECT v.i from (SELECT i FROM v)`. Now this query will throw analysis exception because users should not be able to use the qualifier inside a subquery.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18559 from cloud-fan/sub-query.
## What changes were proposed in this pull request?
Rename org.apache.spark.sql.catalyst.plans.logical.statsEstimation.Range to ValueInterval.
The current naming is identical to logical operator "range".
Refactoring it to ValueInterval is more accurate.
## How was this patch tested?
unit test
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Wang Gengliang <ltnwgl@gmail.com>
Closes#18549 from gengliangwang/ValueInterval.
## What changes were proposed in this pull request?
Currently we can't produce a `Dataset` containing `Set` in SparkSQL. This PR tries to support serialization/deserialization of `Set`.
Because there's no corresponding internal data type in SparkSQL for a `Set`, the most proper choice for serializing a set should be an array.
## How was this patch tested?
Added unit tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18416 from viirya/SPARK-21204.
## What changes were proposed in this pull request?
When data type is struct, InSet now uses TypeUtils.getInterpretedOrdering (similar to EqualTo) to build a TreeSet. In other cases it will use a HashSet as before (which should be faster). Similarly, In.eval uses Ordering.equiv instead of equals.
## How was this patch tested?
New test in SQLQuerySuite.
Author: Bogdan Raducanu <bogdan@databricks.com>
Closes#18455 from bogdanrdc/SPARK-21228.
## What changes were proposed in this pull request?
Add missing test cases back and revise code style
Follow up the previous PR: https://github.com/apache/spark/pull/18479
## How was this patch tested?
Unit test
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Wang Gengliang <ltnwgl@gmail.com>
Closes#18548 from gengliangwang/stat_propagation_revise.
## What changes were proposed in this pull request?
Corrects offsetInBytes calculation in UnsafeRow.writeToStream. Known failures include writes to some DataSources that have own SparkPlan implementations and cause EXCHANGE in writes.
## How was this patch tested?
Extended UnsafeRowSuite.writeToStream to include an UnsafeRow over byte array having non-zero offset.
Author: Sumedh Wale <swale@snappydata.io>
Closes#18535 from sumwale/SPARK-21312.
### What changes were proposed in this pull request?
This PR removes SQLConf parameters from the optimizer rules
### How was this patch tested?
The existing test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18533 from gatorsmile/rmSQLConfOptimizer.
### What changes were proposed in this pull request?
This PR is to remove SQLConf parameters from the parser-related classes.
### How was this patch tested?
The existing test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18531 from gatorsmile/rmSQLConfParser.
## What changes were proposed in this pull request?
support to create [temporary] function with the keyword 'OR REPLACE' and 'IF NOT EXISTS'
## How was this patch tested?
manual test and added test cases
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn>
Closes#17681 from ouyangxiaochen/spark-419.
## What changes were proposed in this pull request?
Currently `RowEncoder` doesn't preserve nullability of `ArrayType` or `MapType`.
It returns always `containsNull = true` for `ArrayType`, `valueContainsNull = true` for `MapType` and also the nullability of itself is always `true`.
This pr fixes the nullability of them.
## How was this patch tested?
Add tests to check if `RowEncoder` preserves array/map nullability.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#13873 from ueshin/issues/SPARK-16167.
## What changes were proposed in this pull request?
Add `returnNullable` to `StaticInvoke` the same as #15780 is trying to add to `Invoke` and modify to handle properly.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#16056 from ueshin/issues/SPARK-18623.
## What changes were proposed in this pull request?
For these collection-related encoder expressions, we don't need to create `isNull` variable if the loop element is not nullable.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18529 from cloud-fan/minor.
## What changes were proposed in this pull request?
`ExternalMapToCatalyst` should null-check map key prior to converting to internal value to throw an appropriate Exception instead of something like NPE.
## How was this patch tested?
Added a test and existing tests.
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#18524 from ueshin/issues/SPARK-21300.
### What changes were proposed in this pull request?
It is strange to see the following error message. Actually, the column is from another table.
```
cannot resolve '`right.a`' given input columns: [a, c, d];
```
After the PR, the error message looks like
```
cannot resolve '`right.a`' given input columns: [left.a, right.c, right.d];
```
### How was this patch tested?
Added a test case
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18520 from gatorsmile/removeSQLConf.
### What changes were proposed in this pull request?
SQLConf is moved to Catalyst. We are adding more and more test cases for verifying the conf-specific behaviors. It is nice to add a helper function to simplify the test cases.
### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18469 from gatorsmile/withSQLConf.
## What changes were proposed in this pull request?
Looking at the code in `SessionCatalog.registerFunction`, the parameter `ignoreIfExists` is a wrong name. When `ignoreIfExists` is true, we will override the function if it already exists. So `overrideIfExists` should be the corrected name.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18510 from cloud-fan/minor.
### Idea
This PR adds validation to REFRESH sql statements. Currently, users can specify whatever they want as resource path. For example, spark.sql("REFRESH ! $ !") will be executed without any exceptions.
### Implementation
I am not sure that my current implementation is the most optimal, so any feedback is appreciated. My first idea was to make the grammar as strict as possible. Unfortunately, there were some problems. I tried the approach below:
SqlBase.g4
```
...
| REFRESH TABLE tableIdentifier #refreshTable
| REFRESH resourcePath #refreshResource
...
resourcePath
: STRING
| (IDENTIFIER | number | nonReserved | '/' | '-')+ // other symbols can be added if needed
;
```
It is not flexible enough and requires to explicitly mention all possible symbols. Therefore, I came up with the current approach that is implemented in the code.
Let me know your opinion on which one is better.
Author: aokolnychyi <anton.okolnychyi@sap.com>
Closes#18368 from aokolnychyi/spark-21102.
## What changes were proposed in this pull request?
We currently implement statistics propagation directly in logical plan. Given we already have two different implementations, it'd make sense to actually decouple the two and add stats propagation using mixin. This would reduce the coupling between logical plan and statistics handling.
This can also be a powerful pattern in the future to add additional properties (e.g. constraints).
## How was this patch tested?
Should be covered by existing test cases.
Author: Reynold Xin <rxin@databricks.com>
Closes#18479 from rxin/stats-trait.
## What changes were proposed in this pull request?
Update stats after the following data changing commands:
- InsertIntoHadoopFsRelationCommand
- InsertIntoHiveTable
- LoadDataCommand
- TruncateTableCommand
- AlterTableSetLocationCommand
- AlterTableDropPartitionCommand
## How was this patch tested?
Added new test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#18334 from wzhfy/changeStatsForOperation.
## What changes were proposed in this pull request?
For performance reasons, `UnsafeRow.getString`, `getStruct`, etc. return a "pointer" that points to a memory region of this unsafe row. This makes the unsafe projection a little dangerous, because all of its output rows share one instance.
When we implement SQL operators, we should be careful to not cache the input rows because they may be produced by unsafe projection from child operator and thus its content may change overtime.
However, when we updating values of InternalRow(e.g. in mutable projection and safe projection), we only copy UTF8String, we should also copy InternalRow, ArrayData and MapData. This PR fixes this, and also fixes the copy of vairous InternalRow, ArrayData and MapData implementations.
## How was this patch tested?
new regression tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18483 from cloud-fan/fix-copy.
### What changes were proposed in this pull request?
Function argument should not be named expressions. It could cause two issues:
- Misleading error message
- Unexpected query results when the column name is `distinct`, which is not a reserved word in our parser.
```
spark-sql> select count(distinct c1, distinct c2) from t1;
Error in query: cannot resolve '`distinct`' given input columns: [c1, c2]; line 1 pos 26;
'Project [unresolvedalias('count(c1#30, 'distinct), None)]
+- SubqueryAlias t1
+- CatalogRelation `default`.`t1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#30, c2#31]
```
After the fix, the error message becomes
```
spark-sql> select count(distinct c1, distinct c2) from t1;
Error in query:
extraneous input 'c2' expecting {')', ',', '.', '[', 'OR', 'AND', 'IN', NOT, 'BETWEEN', 'LIKE', RLIKE, 'IS', EQ, '<=>', '<>', '!=', '<', LTE, '>', GTE, '+', '-', '*', '/', '%', 'DIV', '&', '|', '||', '^'}(line 1, pos 35)
== SQL ==
select count(distinct c1, distinct c2) from t1
-----------------------------------^^^
```
### How was this patch tested?
Added a test case to parser suite.
Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18338 from gatorsmile/parserDistinctAggFunc.
## What changes were proposed in this pull request?
Invalidate spark's stats after data changing commands:
- InsertIntoHadoopFsRelationCommand
- InsertIntoHiveTable
- LoadDataCommand
- TruncateTableCommand
- AlterTableSetLocationCommand
- AlterTableDropPartitionCommand
## How was this patch tested?
Added test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#18449 from wzhfy/removeStats.
## What changes were proposed in this pull request?
`QueryPlan.preCanonicalized` is only overridden in a few places, and it does introduce an extra concept to `QueryPlan` which may confuse people.
This PR removes it and override `canonicalized` in these places
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18440 from cloud-fan/minor.
## What changes were proposed in this pull request?
Move elimination of Distinct clause from analyzer to optimizer
Distinct clause is useless after MAX/MIN clause. For example,
"Select MAX(distinct a) FROM src from"
is equivalent of
"Select MAX(a) FROM src from"
However, this optimization is implemented in analyzer. It should be in optimizer.
## How was this patch tested?
Unit test
gatorsmile cloud-fan
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Wang Gengliang <ltnwgl@gmail.com>
Closes#18429 from gengliangwang/distinct_opt.
## What changes were proposed in this pull request?
The issue happens in `ExternalMapToCatalyst`. For example, the following codes create `ExternalMapToCatalyst` to convert Scala Map to catalyst map format.
val data = Seq.tabulate(10)(i => NestedData(1, Map("key" -> InnerData("name", i + 100))))
val ds = spark.createDataset(data)
The `valueConverter` in `ExternalMapToCatalyst` looks like:
if (isnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true))) null else named_struct(name, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).name, true), value, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).value)
There is a `CreateNamedStruct` expression (`named_struct`) to create a row of `InnerData.name` and `InnerData.value` that are referred by `ExternalMapToCatalyst_value52`.
Because `ExternalMapToCatalyst_value52` are local variable, when `CreateNamedStruct` splits expressions to individual functions, the local variable can't be accessed anymore.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18418 from viirya/SPARK-19104.
## What changes were proposed in this pull request?
Time windowing in Spark currently performs an Expand + Filter, because there is no way to guarantee the amount of windows a timestamp will fall in, in the general case. However, for tumbling windows, a record is guaranteed to fall into a single bucket. In this case, doubling the number of records with Expand is wasteful, and can be improved by using a simple Projection instead.
Benchmarks show that we get an order of magnitude performance improvement after this patch.
## How was this patch tested?
Existing unit tests. Benchmarked using the following code:
```scala
import org.apache.spark.sql.functions._
spark.time {
spark.range(numRecords)
.select(from_unixtime((current_timestamp().cast("long") * 1000 + 'id / 1000) / 1000) as 'time)
.select(window('time, "10 seconds"))
.count()
}
```
Setup:
- 1 c3.2xlarge worker (8 cores)
![image](https://user-images.githubusercontent.com/5243515/27348748-ed991b84-55a9-11e7-8f8b-6e7abc524417.png)
1 B rows ran in 287 seconds after this optimization. I didn't wait for it to finish without the optimization. Shows about 5x improvement for large number of records.
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#18364 from brkyvz/opt-tumble.
### What changes were proposed in this pull request?
```SQL
CREATE TABLE `tab1`
(`custom_fields` ARRAY<STRUCT<`id`: BIGINT, `value`: STRING>>)
USING parquet
INSERT INTO `tab1`
SELECT ARRAY(named_struct('id', 1, 'value', 'a'), named_struct('id', 2, 'value', 'b'))
SELECT custom_fields.id, custom_fields.value FROM tab1
```
The above query always return the last struct of the array, because the rule `SimplifyCasts` incorrectly rewrites the query. The underlying cause is we always use the same `GenericInternalRow` object when doing the cast.
### How was this patch tested?
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18412 from gatorsmile/castStruct.
## What changes were proposed in this pull request?
`isTableSample` and `isGenerated ` were introduced for SQL Generation respectively by https://github.com/apache/spark/pull/11148 and https://github.com/apache/spark/pull/11050
Since SQL Generation is removed, we do not need to keep `isTableSample`.
## How was this patch tested?
The existing test cases
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18379 from gatorsmile/CleanSample.
## What changes were proposed in this pull request?
Currently we do a lot of validations for subquery in the Analyzer. We should move them to CheckAnalysis which is the framework to catch and report Analysis errors. This was mentioned as a review comment in SPARK-18874.
## How was this patch tested?
Exists tests + A few tests added to SQLQueryTestSuite.
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#17713 from dilipbiswal/subquery_checkanalysis.
## What changes were proposed in this pull request?
If the SQL conf for StateStore provider class is changed between restarts (i.e. query started with providerClass1 and attempted to restart using providerClass2), then the query will fail in a unpredictable way as files saved by one provider class cannot be used by the newer one.
Ideally, the provider class used to start the query should be used to restart the query, and the configuration in the session where it is being restarted should be ignored.
This PR saves the provider class config to OffsetSeqLog, in the same way # shuffle partitions is saved and recovered.
## How was this patch tested?
new unit tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#18402 from tdas/SPARK-21192.
## What changes were proposed in this pull request?
After wiring `SQLConf` in logical plan ([PR 18299](https://github.com/apache/spark/pull/18299)), we can remove the need of passing `conf` into `def stats` and `def computeStats`.
## How was this patch tested?
Covered by existing tests, plus some modified existing tests.
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#18391 from wzhfy/removeConf.
## What changes were proposed in this pull request?
The current master outputs unexpected results when the data schema and partition schema have the duplicate columns:
```
withTempPath { dir =>
val basePath = dir.getCanonicalPath
spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=1").toString)
spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=a").toString)
spark.read.parquet(basePath).show()
}
+---+
|foo|
+---+
| 1|
| 1|
| a|
| a|
| 1|
| a|
+---+
```
This patch added code to print a warning when the duplication found.
## How was this patch tested?
Manually checked.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18375 from maropu/SPARK-21144-3.
## What changes were proposed in this pull request?
Currently the validation of sampling fraction in dataset is incomplete.
As an improvement, validate sampling fraction in logical operator level:
1) if with replacement: fraction should be nonnegative
2) else: fraction should be on interval [0, 1]
Also add test cases for the validation.
## How was this patch tested?
integration tests
gatorsmile cloud-fan
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Wang Gengliang <ltnwgl@gmail.com>
Closes#18387 from gengliangwang/sample_ratio_validate.
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`. This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process. The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame. All non-complex data types are currently supported, otherwise an `UnsupportedOperation` exception is thrown.
Additions to Spark include a Scala package private method `Dataset.toArrowPayloadBytes` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served. A package private class/object `ArrowConverters` that provide data type mappings and conversion routines. In Python, a public method `DataFrame.collectAsArrow` is added to collect Arrow payloads and an optional flag in `toPandas(useArrow=False)` to enable using Arrow (uses the old conversion by default).
## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types. The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data. This will ensure that the schema and data has been converted correctly.
Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow. A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.
Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>
Closes#15821 from BryanCutler/wip-toPandas_with_arrow-SPARK-13534.
## What changes were proposed in this pull request?
Currently we have several tens of test sqls in catalyst will fail at `SimpleAnalyzer.checkAnalysis`, we should make sure they are valid.
This PR makes the following changes:
1. Apply `checkAnalysis` on plans that tests `Optimizer` rules, but don't require the testcases for `Parser`/`Analyzer` pass `checkAnalysis`;
2. Fix testcases for `Optimizer` that would have fall.
## How was this patch tested?
Apply `SimpleAnalyzer.checkAnalysis` on plans in `PlanTest.comparePlans`, update invalid test cases.
Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15417 from jiangxb1987/cptest.
## What changes were proposed in this pull request?
QueryPlanConstraints should be part of LogicalPlan, rather than QueryPlan, since the constraint framework is only used for query plan rewriting and not for physical planning.
## How was this patch tested?
Should be covered by existing tests, since it is a simple refactoring.
Author: Reynold Xin <rxin@databricks.com>
Closes#18310 from rxin/SPARK-21103.
## What changes were proposed in this pull request?
Fix some typo of the document.
## How was this patch tested?
Existing tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Xianyang Liu <xianyang.liu@intel.com>
Closes#18350 from ConeyLiu/fixtypo.
## What changes were proposed in this pull request?
This PR cleans up a few Java linter errors for Apache Spark 2.2 release.
## How was this patch tested?
```bash
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```
We can check the result at Travis CI, [here](https://travis-ci.org/dongjoon-hyun/spark/builds/244297894).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#18345 from dongjoon-hyun/fix_lint_java_2.
### What changes were proposed in this pull request?
We should not silently ignore `DISTINCT` when they are not supported in the function arguments. This PR is to block these cases and issue the error messages.
### How was this patch tested?
Added test cases for both regular functions and window functions
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18340 from gatorsmile/firstCount.
## What changes were proposed in this pull request?
Built-in SQL Function UnaryMinus/UnaryPositive support string type, if it's string type, convert it to double type, after this PR:
```sql
spark-sql> select positive('-1.11'), negative('-1.11');
-1.11 1.11
spark-sql>
```
## How was this patch tested?
unit tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18173 from wangyum/SPARK-20948.
## What changes were proposed in this pull request?
This PR adds built-in SQL function `BIT_LENGTH()`, `CHAR_LENGTH()`, and `OCTET_LENGTH()` functions.
`BIT_LENGTH()` returns the bit length of the given string or binary expression.
`CHAR_LENGTH()` returns the length of the given string or binary expression. (i.e. equal to `LENGTH()`)
`OCTET_LENGTH()` returns the byte length of the given string or binary expression.
## How was this patch tested?
Added new test suites for these three functions
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#18046 from kiszk/SPARK-20749.
## What changes were proposed in this pull request?
This pull-request exclusively includes the class splitting feature described in #16648. When code for a given class would grow beyond 1600k bytes, a private, nested sub-class is generated into which subsequent functions are inlined. Additional sub-classes are generated as the code threshold is met subsequent times. This code includes 3 changes:
1. Includes helper maps, lists, and functions for keeping track of sub-classes during code generation (included in the `CodeGenerator` class). These helper functions allow nested classes and split functions to be initialized/declared/inlined to the appropriate locations in the various projection classes.
2. Changes `addNewFunction` to return a string to support instances where a split function is inlined to a nested class and not the outer class (and so must be invoked using the class-qualified name). Uses of `addNewFunction` throughout the codebase are modified so that the returned name is properly used.
3. Removes instances of the `this` keyword when used on data inside generated classes. All state declared in the outer class is by default global and accessible to the nested classes. However, if a reference to global state in a nested class is prepended with the `this` keyword, it would attempt to reference state belonging to the nested class (which would not exist), rather than the correct variable belonging to the outer class.
## How was this patch tested?
Added a test case to the `GeneratedProjectionSuite` that increases the number of columns tested in various projections to a threshold that would previously have triggered a `JaninoRuntimeException` for the Constant Pool.
Note: This PR does not address the second Constant Pool issue with code generation (also mentioned in #16648): excess global mutable state. A second PR may be opened to resolve that issue.
Author: ALeksander Eskilson <alek.eskilson@cerner.com>
Closes#18075 from bdrillard/class_splitting_only.
### What changes were proposed in this pull request?
The current option name `wholeFile` is misleading for CSV users. Currently, it is not representing a record per file. Actually, one file could have multiple records. Thus, we should rename it. Now, the proposal is `multiLine`.
### How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18202 from gatorsmile/renameCVSOption.
## What changes were proposed in this pull request?
It is really painful to not have configs in logical plan and expressions. We had to add all sorts of hacks (e.g. pass SQLConf explicitly in functions). This patch exposes SQLConf in logical plan, using a thread local variable and a getter closure that's set once there is an active SparkSession.
The implementation is a bit of a hack, since we didn't anticipate this need in the beginning (config was only exposed in physical plan). The implementation is described in `SQLConf.get`.
In terms of future work, we should follow up to clean up CBO (remove the need for passing in config).
## How was this patch tested?
Updated relevant tests for constraint propagation.
Author: Reynold Xin <rxin@databricks.com>
Closes#18299 from rxin/SPARK-21092.
## What changes were proposed in this pull request?
This patch moves constraint related code into a separate trait QueryPlanConstraints, so we don't litter QueryPlan with a lot of constraint private functions.
## How was this patch tested?
This is a simple move refactoring and should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#18298 from rxin/SPARK-21091.
### What changes were proposed in this pull request?
Since both table properties and storage properties share the same key values, table properties are not shown in the output of DESC EXTENDED/FORMATTED when the storage properties are not empty.
This PR is to fix the above issue by renaming them to different keys.
### How was this patch tested?
Added test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18294 from gatorsmile/tableProperties.
## What changes were proposed in this pull request?
Since `stack` function generates a table with nullable columns, it should allow mixed null values.
```scala
scala> sql("select stack(3, 1, 2, 3)").printSchema
root
|-- col0: integer (nullable = true)
scala> sql("select stack(3, 1, 2, null)").printSchema
org.apache.spark.sql.AnalysisException: cannot resolve 'stack(3, 1, 2, NULL)' due to data type mismatch: Argument 1 (IntegerType) != Argument 3 (NullType); line 1 pos 7;
```
## How was this patch tested?
Pass the Jenkins with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#17251 from dongjoon-hyun/SPARK-19910.
## What changes were proposed in this pull request?
This patch fixes a bug that can cause NullPointerException in LikeSimplification, when the pattern for like is null.
## How was this patch tested?
Added a new unit test case in LikeSimplificationSuite.
Author: Reynold Xin <rxin@databricks.com>
Closes#18273 from rxin/SPARK-21059.
The PR contains a tiny change to fix the way Spark parses string literals into timestamps. Currently, some timestamps that contain nanoseconds are corrupted during the conversion from internal UTF8Strings into the internal representation of timestamps.
Consider the following example:
```
spark.sql("SELECT cast('2015-01-02 00:00:00.000000001' as TIMESTAMP)").show(false)
+------------------------------------------------+
|CAST(2015-01-02 00:00:00.000000001 AS TIMESTAMP)|
+------------------------------------------------+
|2015-01-02 00:00:00.000001 |
+------------------------------------------------+
```
The fix was tested with existing tests. Also, there is a new test to cover cases that did not work previously.
Author: aokolnychyi <anton.okolnychyi@sap.com>
Closes#18252 from aokolnychyi/spark-17914.
## What changes were proposed in this pull request?
add test case to MathExpressionsSuite as #17906
## How was this patch tested?
unit test cases
Author: liuxian <liu.xian3@zte.com.cn>
Closes#18082 from 10110346/wip-lx-0524.
## What changes were proposed in this pull request?
Add support for specific Java `List` subtypes in deserialization as well as a generic implicit encoder.
All `List` subtypes are supported by using either the size-specifying constructor (one `int` parameter) or the default constructor.
Interfaces/abstract classes use the following implementations:
* `java.util.List`, `java.util.AbstractList` or `java.util.AbstractSequentialList` => `java.util.ArrayList`
## How was this patch tested?
```bash
build/mvn -DskipTests clean package && dev/run-tests
```
Additionally in Spark shell:
```
scala> val jlist = new java.util.LinkedList[Int]; jlist.add(1)
jlist: java.util.LinkedList[Int] = [1]
res0: Boolean = true
scala> Seq(jlist).toDS().map(_.element()).collect()
res1: Array[Int] = Array(1)
```
Author: Michal Senkyr <mike.senkyr@gmail.com>
Closes#18009 from michalsenkyr/dataset-java-lists.
## What changes were proposed in this pull request?
Currently, hive's stats are read into `CatalogStatistics`, while spark's stats are also persisted through `CatalogStatistics`. As a result, hive's stats can be unexpectedly propagated into spark' stats.
For example, for a catalog table, we read stats from hive, e.g. "totalSize" and put it into `CatalogStatistics`. Then, by using "ALTER TABLE" command, we will store the stats in `CatalogStatistics` into metastore as spark's stats (because we don't know whether it's from spark or not). But spark's stats should be only generated by "ANALYZE" command. This is unexpected from this command.
Secondly, now that we have spark's stats in metastore, after inserting new data, although hive updated "totalSize" in metastore, we still cannot get the right `sizeInBytes` in `CatalogStatistics`, because we respect spark's stats (should not exist) over hive's stats.
A running example is shown in [JIRA](https://issues.apache.org/jira/browse/SPARK-21031).
To fix this, we add a new method `alterTableStats` to store spark's stats, and let `alterTable` keep existing stats.
## How was this patch tested?
Added new tests.
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#18248 from wzhfy/separateHiveStats.
## What changes were proposed in this pull request?
add more datatype for some unit tests
## How was this patch tested?
unit tests
Author: liuxian <liu.xian3@zte.com.cn>
Closes#17880 from 10110346/wip_lx_0506.
### What changes were proposed in this pull request?
The precision and scale of decimal values are wrong when the input is BigDecimal between -1.0 and 1.0.
The BigDecimal's precision is the digit count starts from the leftmost nonzero digit based on the [JAVA's BigDecimal definition](https://docs.oracle.com/javase/7/docs/api/java/math/BigDecimal.html). However, our Decimal decision follows the database decimal standard, which is the total number of digits, including both to the left and the right of the decimal point. Thus, this PR is to fix the issue by doing the conversion.
Before this PR, the following queries failed:
```SQL
select 1 > 0.0001
select floor(0.0001)
select ceil(0.0001)
```
### How was this patch tested?
Added test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18244 from gatorsmile/bigdecimal.
### What changes were proposed in this pull request?
Currently, the unquoted string of a function identifier is being used as the function identifier in the function registry. This could cause the incorrect the behavior when users use `.` in the function names. This PR is to take the `FunctionIdentifier` as the identifier in the function registry.
- Add one new function `createOrReplaceTempFunction` to `FunctionRegistry`
```Scala
final def createOrReplaceTempFunction(name: String, builder: FunctionBuilder): Unit
```
### How was this patch tested?
Add extra test cases to verify the inclusive bug fixes.
Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18142 from gatorsmile/fuctionRegistry.
### What changes were proposed in this pull request?
Before 2.2, we indicate the job was terminated because of `FAILFAST` mode.
```
Malformed line in FAILFAST mode: {"a":{, b:3}
```
If possible, we should keep it. This PR is to unify the error messages.
### How was this patch tested?
Modified the existing messages.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18196 from gatorsmile/messFailFast.
## What changes were proposed in this pull request?
Removed a duplicate case in "SPARK-20854: select hint syntax with expressions"
## How was this patch tested?
Existing tests.
Author: Bogdan Raducanu <bogdan@databricks.com>
Closes#18217 from bogdanrdc/SPARK-20854-2.
## What changes were proposed in this pull request?
`HintInfo.isBroadcastable` is actually not an accurate name, it's used to force the planner to broadcast a plan no matter what the data size is, via the hint mechanism. I think `forceBroadcast` is a better name.
And `isBroadcastable` only have 2 possible values: `Some(true)` and `None`, so we can just use boolean type for it.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18189 from cloud-fan/stats.
There could be test failures because DataStorageStrategy, HiveMetastoreCatalog and also HiveSchemaInferenceSuite were exposed to guava library by directly accessing SessionCatalog's tableRelationCacheg. These failures occur when guava shading is in place.
## What changes were proposed in this pull request?
This change removes those guava exposures by introducing new methods in SessionCatalog and also changing DataStorageStrategy, HiveMetastoreCatalog and HiveSchemaInferenceSuite so that they use those proxy methods.
## How was this patch tested?
Unit tests passed after applying these changes.
Author: Reza Safi <rezasafi@cloudera.com>
Closes#18148 from rezasafi/branch-2.2.
(cherry picked from commit 1388fdd707)
## What changes were proposed in this pull request?
The construction of BROADCAST_TIMEOUT conf should take the TimeUnit argument as a TimeoutConf.
Author: Feng Liu <fengliu@databricks.com>
Closes#18208 from liufengdb/fix_timeout.
## What changes were proposed in this pull request?
Fixes a typo: `and` -> `an`
## How was this patch tested?
Not at all.
Author: Wieland Hoffmann <mineo@users.noreply.github.com>
Closes#17759 from mineo/patch-1.
### What changes were proposed in this pull request?
1. The description of `spark.sql.files.ignoreCorruptFiles` is not accurate. When the file does not exist, we will issue the error message.
```
org.apache.spark.sql.AnalysisException: Path does not exist: file:/nonexist/path;
```
2. `spark.sql.columnNameOfCorruptRecord` also affects the CSV format. The current description only mentions JSON format.
### How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18184 from gatorsmile/updateMessage.
## What changes were proposed in this pull request?
SQL hint syntax:
* support expressions such as strings, numbers, etc. instead of only identifiers as it is currently.
* support multiple hints, which was missing compared to the DataFrame syntax.
DataFrame API:
* support any parameters in DataFrame.hint instead of just strings
## How was this patch tested?
Existing tests. New tests in PlanParserSuite. New suite DataFrameHintSuite.
Author: Bogdan Raducanu <bogdan@databricks.com>
Closes#18086 from bogdanrdc/SPARK-20854.
### What changes were proposed in this pull request?
Before this PR, Subquery reuse does not work. Below are three issues:
- Subquery reuse does not work.
- It is sharing the same `SQLConf` (`spark.sql.exchange.reuse`) with the one for Exchange Reuse.
- No test case covers the rule Subquery reuse.
This PR is to fix the above three issues.
- Ignored the physical operator `SubqueryExec` when comparing two plans.
- Added a dedicated conf `spark.sql.subqueries.reuse` for controlling Subquery Reuse
- Added a test case for verifying the behavior
### How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18169 from gatorsmile/subqueryReuse.
## What changes were proposed in this pull request?
Add build-int SQL function - UUID.
## How was this patch tested?
unit tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18136 from wangyum/SPARK-20910.
## What changes were proposed in this pull request?
Minor changes to scaladoc
## How was this patch tested?
Local build
Author: Jacek Laskowski <jacek@japila.pl>
Closes#18074 from jaceklaskowski/scaladoc-fixes.
## What changes were proposed in this pull request?
Currently the `DataFrameWriter` operations have several problems:
1. non-file-format data source writing action doesn't show up in the SQL tab in Spark UI
2. file-format data source writing action shows a scan node in the SQL tab, without saying anything about writing. (streaming also have this issue, but not fixed in this PR)
3. Spark SQL CLI actions don't show up in the SQL tab.
This PR fixes all of them, by refactoring the `ExecuteCommandExec` to make it have children.
close https://github.com/apache/spark/pull/17540
## How was this patch tested?
existing tests.
Also test the UI manually. For a simple command: `Seq(1 -> "a").toDF("i", "j").write.parquet("/tmp/qwe")`
before this PR:
<img width="266" alt="qq20170523-035840 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326050/24e18ba2-3f6c-11e7-8817-6dd275bf6ac5.png">
after this PR:
<img width="287" alt="qq20170523-035708 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326054/2ad7f460-3f6c-11e7-8053-d68325beb28f.png">
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18064 from cloud-fan/execution.
## What changes were proposed in this pull request?
A bunch of changes to the StateStore APIs and implementation.
Current state store API has a bunch of problems that causes too many transient objects causing memory pressure.
- `StateStore.get(): Option` forces creation of Some/None objects for every get. Changed this to return the row or null.
- `StateStore.iterator(): (UnsafeRow, UnsafeRow)` forces creation of new tuple for each record returned. Changed this to return a UnsafeRowTuple which can be reused across records.
- `StateStore.updates()` requires the implementation to keep track of updates, while this is used minimally (only by Append mode in streaming aggregations). Removed updates() and updated StateStoreSaveExec accordingly.
- `StateStore.filter(condition)` and `StateStore.remove(condition)` has been merge into a single API `getRange(start, end)` which allows a state store to do optimized range queries (i.e. avoid full scans). Stateful operators have been updated accordingly.
- Removed a lot of unnecessary row copies Each operator copied rows before calling StateStore.put() even if the implementation does not require it to be copied. It is left up to the implementation on whether to copy the row or not.
Additionally,
- Added a name to the StateStoreId so that each operator+partition can use multiple state stores (different names)
- Added a configuration that allows the user to specify which implementation to use.
- Added new metrics to understand the time taken to update keys, remove keys and commit all changes to the state store. These metrics will be visible on the plan diagram in the SQL tab of the UI.
- Refactored unit tests such that they can be reused to test any implementation of StateStore.
## How was this patch tested?
Old and new unit tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#18107 from tdas/SPARK-20376.
### What changes were proposed in this pull request?
We are unable to call the function registered in the not-current database.
```Scala
sql("CREATE DATABASE dAtABaSe1")
sql(s"CREATE FUNCTION dAtABaSe1.test_avg AS '${classOf[GenericUDAFAverage].getName}'")
sql("SELECT dAtABaSe1.test_avg(1)")
```
The above code returns an error:
```
Undefined function: 'dAtABaSe1.test_avg'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7
```
This PR is to fix the above issue.
### How was this patch tested?
Added test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18146 from gatorsmile/qualifiedFunction.
## What changes were proposed in this pull request?
We changed the parser to reject unaliased subqueries in the FROM clause in SPARK-20690. However, the error message that we now give isn't very helpful:
scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
org.apache.spark.sql.catalyst.parser.ParseException:
mismatched input 'FROM' expecting {<EOF>, 'WHERE', 'GROUP', 'ORDER', 'HAVING', 'LIMIT', 'LATERAL', 'WINDOW', 'UNION', 'EXCEPT', 'MINUS', 'INTERSECT', 'SORT', 'CLUSTER', 'DISTRIBUTE'}(line 1, pos 9)
We should modify the parser to throw a more clear error for such queries:
scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
org.apache.spark.sql.catalyst.parser.ParseException:
The unaliased subqueries in the FROM clause are not supported.(line 1, pos 14)
## How was this patch tested?
Modified existing tests to reflect this change.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18141 from viirya/SPARK-20916.
## What changes were proposed in this pull request?
Fix some indent issues.
## How was this patch tested?
existing tests.
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18133 from wangyum/IndentIssues.
## What changes were proposed in this pull request?
Add build-int SQL function - DAYOFWEEK
## How was this patch tested?
unit tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18134 from wangyum/SPARK-20909.
## What changes were proposed in this pull request?
This PR adds built-in SQL function `(REPLACE(<string_expression>, <search_string> [, <replacement_string>])`
`REPLACE()` return that string that is replaced all occurrences with given string.
## How was this patch tested?
added new test suites
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#18047 from kiszk/SPARK-20750.
## What changes were proposed in this pull request?
See class doc of `ConstantPropagation` for the approach used.
## How was this patch tested?
- Added unit tests
Author: Tejas Patil <tejasp@fb.com>
Closes#17993 from tejasapatil/SPARK-20758_const_propagation.
## What changes were proposed in this pull request?
This pr added parsing rules to support table column aliases in FROM clause.
## How was this patch tested?
Added tests in `PlanParserSuite`, `SQLQueryTestSuite`, and `PlanParserSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18079 from maropu/SPARK-20841.
### What changes were proposed in this pull request?
In Cache manager, the plan matching should ignore Hint.
```Scala
val df1 = spark.range(10).join(broadcast(spark.range(10)))
df1.cache()
spark.range(10).join(spark.range(10)).explain()
```
The output plan of the above query shows that the second query is not using the cached data of the first query.
```
BroadcastNestedLoopJoin BuildRight, Inner
:- *Range (0, 10, step=1, splits=2)
+- BroadcastExchange IdentityBroadcastMode
+- *Range (0, 10, step=1, splits=2)
```
After the fix, the plan becomes
```
InMemoryTableScan [id#20L, id#23L]
+- InMemoryRelation [id#20L, id#23L], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
+- BroadcastNestedLoopJoin BuildRight, Inner
:- *Range (0, 10, step=1, splits=2)
+- BroadcastExchange IdentityBroadcastMode
+- *Range (0, 10, step=1, splits=2)
```
### How was this patch tested?
Added a test.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18131 from gatorsmile/HintCache.
## What changes were proposed in this pull request?
spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12345
For this case, the result we expected is `12346`
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12345
For this case, the result we expected is `-12346`
Because in `Ceil` or `Floor`, `inputTypes` has no FloatType, so it is converted to LongType.
## How was this patch tested?
After the modification:
spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12346
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12346
Author: liuxian <liu.xian3@zte.com.cn>
Closes#18103 from 10110346/wip-lx-0525-1.
## What changes were proposed in this pull request?
Add built-in SQL function `CH[A]R`:
For `CHR(bigint|double n)`, returns the ASCII character having the binary equivalent to `n`. If n is larger than 256 the result is equivalent to CHR(n % 256)
## How was this patch tested?
unit tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18019 from wangyum/SPARK-20748.
Now that Structured Streaming has been out for several Spark release and has large production use cases, the `Experimental` label is no longer appropriate. I've left `InterfaceStability.Evolving` however, as I think we may make a few changes to the pluggable Source & Sink API in Spark 2.3.
Author: Michael Armbrust <michael@databricks.com>
Closes#18065 from marmbrus/streamingGA.
## What changes were proposed in this pull request?
It is reported that there is performance downgrade when applying ML pipeline for dataset with many columns but few rows.
A big part of the performance downgrade comes from some operations (e.g., `select`) on DataFrame/Dataset which re-create new DataFrame/Dataset with a new `LogicalPlan`. The cost can be ignored in the usage of SQL, normally.
However, it's not rare to chain dozens of pipeline stages in ML. When the query plan grows incrementally during running those stages, the total cost spent on re-creation of DataFrame grows too. In particular, the `Analyzer` will go through the big query plan even most part of it is analyzed.
By eliminating part of the cost, the time to run the example code locally is reduced from about 1min to about 30 secs.
In particular, the time applying the pipeline locally is mostly spent on calling transform of the 137 `Bucketizer`s. Before the change, each call of `Bucketizer`'s transform can cost about 0.4 sec. So the total time spent on all `Bucketizer`s' transform is about 50 secs. After the change, each call only costs about 0.1 sec.
<del>We also make `boundEnc` as lazy variable to reduce unnecessary running time.</del>
### Performance improvement
The codes and datasets provided by Barry Becker to re-produce this issue and benchmark can be found on the JIRA.
Before this patch: about 1 min
After this patch: about 20 secs
## How was this patch tested?
Existing tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#17770 from viirya/SPARK-20392.
## What changes were proposed in this pull request?
1. add instructions of 'cast' function When using 'show functions' and 'desc function cast'
command in spark-sql
2. Modify the instructions of functions,such as
boolean,tinyint,smallint,int,bigint,float,double,decimal,date,timestamp,binary,string
## How was this patch tested?
Before modification:
spark-sql>desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr AS type) - Casts the value `expr` to the target data type `type`.
After modification:
spark-sql> desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`.
spark-sql> desc function cast
Function: cast
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: cast(expr AS type) - Casts the value `expr` to the target data type `type`.
Author: liuxian <liu.xian3@zte.com.cn>
Closes#17698 from 10110346/wip_lx_0418.
## What changes were proposed in this pull request?
This is a follow-up to SPARK-20857 to move the broadcast hint from Statistics into a new HintInfo class, so we can be more flexible in adding new hints in the future.
## How was this patch tested?
Updated test cases to reflect the change.
Author: Reynold Xin <rxin@databricks.com>
Closes#18087 from rxin/SPARK-20867.
## What changes were proposed in this pull request?
This patch renames BroadcastHint to ResolvedHint (and Hint to UnresolvedHint) so the hint framework is more generic and would allow us to introduce other hint types in the future without introducing new hint nodes.
## How was this patch tested?
Updated test cases.
Author: Reynold Xin <rxin@databricks.com>
Closes#18072 from rxin/SPARK-20857.
## What changes were proposed in this pull request?
As srowen pointed in 609ba5f2b9 (commitcomment-22221259), the previous tests are not proper.
This follow-up is going to fix the tests.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18048 from viirya/SPARK-20399-follow-up.
### What changes were proposed in this pull request?
After we adding a new field `stats` into `CatalogTable`, we should not expose Hive-specific Stats metadata to `MetastoreRelation`. It complicates all the related codes. It also introduces a bug in `SHOW CREATE TABLE`. The statistics-related table properties should be skipped by `SHOW CREATE TABLE`, since it could be incorrect in the newly created table. See the Hive JIRA: https://issues.apache.org/jira/browse/HIVE-13792
Also fix the issue to fill Hive-generated RowCounts to our stats.
This PR is to handle Hive-specific Stats metadata in `HiveClientImpl`.
### How was this patch tested?
Added a few test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#14971 from gatorsmile/showCreateTableNew.
### What changes were proposed in this pull request?
Currently, we have a bug when we specify `IF NOT EXISTS` in `INSERT OVERWRITE` data source tables. For example, given a query:
```SQL
INSERT OVERWRITE TABLE $tableName partition (b=2, c=3) IF NOT EXISTS SELECT 9, 10
```
we will get the following error:
```
unresolved operator 'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true;;
'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true
+- Project [cast(9#423 as int) AS a#429, cast(10#424 as int) AS d#430]
+- Project [9 AS 9#423, 10 AS 10#424]
+- OneRowRelation$
```
This PR is to fix the issue to follow the behavior of Hive serde tables
> INSERT OVERWRITE will overwrite any existing data in the table or partition unless IF NOT EXISTS is provided for a partition
### How was this patch tested?
Modified an existing test case
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18050 from gatorsmile/insertPartitionIfNotExists.
## What changes were proposed in this pull request?
spark-sql>SELECT ceil(1234567890123456);
1234567890123456
spark-sql>SELECT ceil(12345678901234567);
12345678901234568
spark-sql>SELECT ceil(123456789012345678);
123456789012345680
when the length of the getText is greater than 16. long to double will be precision loss.
but mysql handle the value is ok.
mysql> SELECT ceil(1234567890123456);
+------------------------+
| ceil(1234567890123456) |
+------------------------+
| 1234567890123456 |
+------------------------+
1 row in set (0.00 sec)
mysql> SELECT ceil(12345678901234567);
+-------------------------+
| ceil(12345678901234567) |
+-------------------------+
| 12345678901234567 |
+-------------------------+
1 row in set (0.00 sec)
mysql> SELECT ceil(123456789012345678);
+--------------------------+
| ceil(123456789012345678) |
+--------------------------+
| 123456789012345678 |
+--------------------------+
1 row in set (0.00 sec)
## How was this patch tested?
Supplement the unit test.
Author: caoxuewen <cao.xuewen@zte.com.cn>
Closes#18016 from heary-cao/ceil_long.
## What changes were proposed in this pull request?
spark-sql>select month("1582-09-28");
spark-sql>10
For this case, the expected result is 9, but it is 10.
spark-sql>select day("1582-04-18");
spark-sql>28
For this case, the expected result is 18, but it is 28.
when the date before "1582-10-04", the function of `month` and `day` return the value which is not we expected.
## How was this patch tested?
unit tests
Author: liuxian <liu.xian3@zte.com.cn>
Closes#17997 from 10110346/wip_lx_0516.
## What changes were proposed in this pull request?
Add built-in SQL Function - COT.
## How was this patch tested?
unit tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#17999 from wangyum/SPARK-20751.
## What changes were proposed in this pull request?
GenerateUnsafeProjection.writeStructToBuffer() did not honor the assumption that the caller must make sure that a value is not null before using the getter. This could lead to various errors. This change fixes that behavior.
Example of code generated before:
```scala
/* 059 */ final UTF8String fieldName = value.getUTF8String(0);
/* 060 */ if (value.isNullAt(0)) {
/* 061 */ rowWriter1.setNullAt(0);
/* 062 */ } else {
/* 063 */ rowWriter1.write(0, fieldName);
/* 064 */ }
```
Example of code generated now:
```scala
/* 060 */ boolean isNull1 = value.isNullAt(0);
/* 061 */ UTF8String value1 = isNull1 ? null : value.getUTF8String(0);
/* 062 */ if (isNull1) {
/* 063 */ rowWriter1.setNullAt(0);
/* 064 */ } else {
/* 065 */ rowWriter1.write(0, value1);
/* 066 */ }
```
## How was this patch tested?
Adds GenerateUnsafeProjectionSuite.
Author: Ala Luszczak <ala@databricks.com>
Closes#18030 from ala/fix-generate-unsafe-projection.
## What changes were proposed in this pull request?
In the previous approach we used `aliasMap` to link an `Attribute` to the expression with potentially the form `f(a, b)`, but we only searched the `expressions` and `children.expressions` for this, which is not enough when an `Alias` may lies deep in the logical plan. In that case, we can't generate the valid equivalent constraint classes and thus we fail at preventing the recursive deductions.
We fix this problem by collecting all `Alias`s from the logical plan.
## How was this patch tested?
No additional test case is added, but do modified one test case to cover this situation.
Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Closes#18020 from jiangxb1987/inferConstrants.
## What changes were proposed in this pull request?
We add missing attributes into Filter in Analyzer. But we shouldn't do it through subqueries like this:
select 1 from (select 1 from onerow t1 LIMIT 1) where t1.c1=1
This query works in current codebase. However, the outside where clause shouldn't be able to refer `t1.c1` attribute.
The root cause is we allow subqueries in FROM have no alias names previously, it is confusing and isn't supported by various databases such as MySQL, Postgres, Oracle. We shouldn't support it too.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#17935 from viirya/SPARK-20690.
## What changes were proposed in this pull request?
Currently the parser logs the query it is parsing at `info` level. This is too high, this PR lowers the log level to `debug`.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#18006 from hvanhovell/lower_parser_log_level.
## What changes were proposed in this pull request?
When an expression for `df.filter()` has many nodes (e.g. 400), the size of Java bytecode for the generated Java code is more than 64KB. It produces an Java exception. As a result, the execution fails.
This PR continues to execute by calling `Expression.eval()` disabling code generation if an exception has been caught.
## How was this patch tested?
Add a test suite into `DataFrameSuite`
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#17087 from kiszk/SPARK-19372.
## What changes were proposed in this pull request?
Because the method `TimeZone.getTimeZone(String ID)` is synchronized on the TimeZone class, concurrent call of this method will become a bottleneck.
This especially happens when casting from string value containing timezone info to timestamp value, which uses `DateTimeUtils.stringToTimestamp()` and gets TimeZone instance on the site.
This pr makes a cache of the generated TimeZone instances to avoid the synchronization.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#17933 from ueshin/issues/SPARK-20588.
## What changes were proposed in this pull request?
Hive allows inserting data to bucketed table without guaranteeing bucketed and sorted-ness based on these two configs : `hive.enforce.bucketing` and `hive.enforce.sorting`.
What does this PR achieve ?
- Spark will disallow users from writing outputs to hive bucketed tables by default (given that output won't adhere with Hive's semantics).
- IF user still wants to write to hive bucketed table, the only resort is to use `hive.enforce.bucketing=false` and `hive.enforce.sorting=false` which means user does NOT care about bucketing guarantees.
Changes done in this PR:
- Extract table's bucketing information in `HiveClientImpl`
- While writing table info to metastore, `HiveClientImpl` now populates the bucketing information in the hive `Table` object
- `InsertIntoHiveTable` allows inserts to bucketed table only if both `hive.enforce.bucketing` and `hive.enforce.sorting` are `false`
Ability to create bucketed tables will enable adding test cases to Spark while I add more changes related to hive bucketing support. Design doc for hive hive bucketing support : https://docs.google.com/document/d/1a8IDh23RAkrkg9YYAeO51F4aGO8-xAlupKwdshve2fc/edit#
## How was this patch tested?
- Added test for creating bucketed and sorted table.
- Added test to ensure that INSERTs fail if strict bucket / sort is enforced
- Added test to ensure that INSERTs can go through if strict bucket / sort is NOT enforced
- Added test to validate that bucketing information shows up in output of DESC FORMATTED
- Added test to ensure that `SHOW CREATE TABLE` works for hive bucketed tables
Author: Tejas Patil <tejasp@fb.com>
Closes#17644 from tejasapatil/SPARK-17729_create_bucketed_table.
## What changes were proposed in this pull request?
This pr added a new Optimizer rule to combine nested Concat. The master supports a pipeline operator '||' to concatenate strings in #17711 (This pr is follow-up). Since the parser currently generates nested Concat expressions, the optimizer needs to combine the nested expressions.
## How was this patch tested?
Added tests in `CombineConcatSuite` and `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17970 from maropu/SPARK-20730.
## What changes were proposed in this pull request?
For aggregate function with `PartialMerge` or `Final` mode, the input is aggregate buffers instead of the actual children expressions. So the actual children expressions won't affect the result, we should normalize the expr id for them.
## How was this patch tested?
a new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17964 from cloud-fan/tmp.
## What changes were proposed in this pull request?
This PR is based on https://github.com/apache/spark/pull/16199 and extracts the valid change from https://github.com/apache/spark/pull/9759 to resolve SPARK-18772
This avoids additional conversion try with `toFloat` and `toDouble`.
For avoiding additional conversions, please refer the codes below:
**Before**
```scala
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._
scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:30:41 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)
java.lang.NumberFormatException: For input string: "nan"
...
```
**After**
```scala
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._
scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:44:30 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.RuntimeException: Cannot parse nan as DoubleType.
...
```
## How was this patch tested?
Unit tests added in `JsonSuite`.
Closes#16199
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Nathan Howell <nhowell@godaddy.com>
Closes#17956 from HyukjinKwon/SPARK-18772.
### What changes were proposed in this pull request?
`LIMIT ALL` is the same as omitting the `LIMIT` clause. It is supported by both PrestgreSQL and Presto. This PR is to support it by adding it in the parser.
### How was this patch tested?
Added a test case
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17960 from gatorsmile/LimitAll.
## What changes were proposed in this pull request?
This pr added code to support `||` for string concatenation. This string operation is supported in PostgreSQL and MySQL.
## How was this patch tested?
Added tests in `SparkSqlParserSuite`
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17711 from maropu/SPARK-19951.
## What changes were proposed in this pull request?
This pr added `Analyzer` code for supporting aliases in CUBE/ROLLUP/GROUPING SETS (This is follow-up of #17191).
## How was this patch tested?
Added tests in `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17948 from maropu/SPARK-20710.
## What changes were proposed in this pull request?
Fix canonicalization for different filter orders in `HiveTableScanExec`.
## How was this patch tested?
Added a new test case.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17962 from wzhfy/canonicalizeHiveTableScanExec.
## What changes were proposed in this pull request?
This method gets a type's primary constructor and fills in type parameters with concrete types. For example, `MapPartitions[T, U] -> MapPartitions[Int, String]`. This Substitution fails when the actual type args are empty because they are still unknown. Instead, when there are no resolved types to subsitute, this returns the original args with unresolved type parameters.
## How was this patch tested?
This doesn't affect substitutions where the type args are determined. This fixes our case where the actual type args are empty and our job runs successfully.
Author: Ryan Blue <blue@apache.org>
Closes#15062 from rdblue/SPARK-17424-fix-unsound-reflect-substitution.
## What changes were proposed in this pull request?
This PR proposes three things as below:
- Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`).
- Support single argument for `to_timestamp` similarly with APIs in other languages.
For example, the one below works
```
import org.apache.spark.sql.functions._
Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show()
```
prints
```
+----------------------------------------+
|to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')|
+----------------------------------------+
| 2016-12-31 00:12:00|
+----------------------------------------+
```
whereas this does not work in SQL.
**Before**
```
spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7
```
**After**
```
spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
2016-12-31 00:12:00
```
- Related document improvement for SQL function descriptions and other API descriptions accordingly.
**Before**
```
spark-sql> DESCRIBE FUNCTION extended to_date;
...
Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.
Extended Usage:
Examples:
> SELECT to_date('2016-12-31', 'yyyy-MM-dd');
2016-12-31
```
```
spark-sql> DESCRIBE FUNCTION extended to_timestamp;
...
Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.
Extended Usage:
Examples:
> SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
2016-12-31 00:00:00.0
```
**After**
```
spark-sql> DESCRIBE FUNCTION extended to_date;
...
Usage:
to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to
a date. Returns null with invalid input. By default, it follows casting rules to a date if
the `fmt` is omitted.
Extended Usage:
Examples:
> SELECT to_date('2009-07-30 04:17:52');
2009-07-30
> SELECT to_date('2016-12-31', 'yyyy-MM-dd');
2016-12-31
```
```
spark-sql> DESCRIBE FUNCTION extended to_timestamp;
...
Usage:
to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to
a timestamp. Returns null with invalid input. By default, it follows casting rules to
a timestamp if the `fmt` is omitted.
Extended Usage:
Examples:
> SELECT to_timestamp('2016-12-31 00:12:00');
2016-12-31 00:12:00
> SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
2016-12-31 00:00:00
```
## How was this patch tested?
Added tests in `datetime.sql`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17901 from HyukjinKwon/to_timestamp_arg.
## What changes were proposed in this pull request?
spark-sql>select bround(12.3, 2);
spark-sql>NULL
For this case, the expected result is 12.3, but it is null.
So ,when the second parameter is bigger than "decimal.scala", the result is not we expected.
"round" function has the same problem. This PR can solve the problem for both of them.
## How was this patch tested?
unit test cases in MathExpressionsSuite and MathFunctionsSuite
Author: liuxian <liu.xian3@zte.com.cn>
Closes#17906 from 10110346/wip_lx_0509.
## What changes were proposed in this pull request?
The new SQL parser is introduced into Spark 2.0. All string literals are unescaped in parser. Seems it bring an issue regarding the regex pattern string.
The following codes can reproduce it:
val data = Seq("\u0020\u0021\u0023", "abc")
val df = data.toDF()
// 1st usage: works in 1.6
// Let parser parse pattern string
val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'")
// 2nd usage: works in 1.6, 2.x
// Call Column.rlike so the pattern string is a literal which doesn't go through parser
val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$"))
// In 2.x, we need add backslashes to make regex pattern parsed correctly
val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'")
Follow the discussion in #17736, this patch adds a config to fallback to 1.6 string literal parsing and mitigate migration issue.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#17887 from viirya/add-config-fallback-string-parsing.
## What changes were proposed in this pull request?
This pr added parsing rules to support aliases in table value functions.
The previous pr (#17666) has been reverted because of the regression. This new pr fixed the regression and add tests in `SQLQueryTestSuite`.
## How was this patch tested?
Added tests in `PlanParserSuite` and `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17928 from maropu/SPARK-20311-3.
## What changes were proposed in this pull request?
`RuntimeReplaceable` always has a constructor with the expression to replace with, and this constructor should not be the function builder.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17876 from cloud-fan/minor.
## What changes were proposed in this pull request?
In filter estimation, we update column stats for those columns in filter condition. However, if the number of rows decreases after the filter (i.e. the overall selectivity is less than 1), we need to update (scale down) the number of distinct values (NDV) for all columns, no matter they are in filter conditions or not.
This pr also fixes the inconsistency of rounding mode for ndv and rowCount.
## How was this patch tested?
Added new tests.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17918 from wzhfy/scaleDownNdvAfterFilter.
## What changes were proposed in this pull request?
In `CheckAnalysis`, we should call `checkAnalysis` for `ScalarSubquery` at the beginning, as later we will call `plan.output` which is invalid if `plan` is not resolved.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17930 from cloud-fan/tmp.
## What changes were proposed in this pull request?
When registering Scala UDF, we can know if the udf will return nullable value or not. `ScalaUDF` and related classes should handle the nullability.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#17911 from ueshin/issues/SPARK-20668.
## What changes were proposed in this pull request?
The query
```
SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1
```
should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows.
This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead:
An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows.
If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation.
The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be).
This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation.
## How was this patch tested?
- Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file.
- Updated unit tests in `PropagateEmptyRelationSuite`.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#17929 from JoshRosen/fix-PropagateEmptyRelation.
## What changes were proposed in this pull request?
Any Dataset/DataFrame batch query with the operation `withWatermark` does not execute because the batch planner does not have any rule to explicitly handle the EventTimeWatermark logical plan.
The right solution is to simply remove the plan node, as the watermark should not affect any batch query in any way.
Changes:
- In this PR, we add a new rule `EliminateEventTimeWatermark` to check if we need to ignore the event time watermark. We will ignore watermark in any batch query.
Depends upon:
- [SPARK-20672](https://issues.apache.org/jira/browse/SPARK-20672). We can not add this rule into analyzer directly, because streaming query will be copied to `triggerLogicalPlan ` in every trigger, and the rule will be applied to `triggerLogicalPlan` mistakenly.
Others:
- A typo fix in example.
## How was this patch tested?
add new unit test.
Author: uncleGen <hustyugm@gmail.com>
Closes#17896 from uncleGen/SPARK-20373.
## What changes were proposed in this pull request?
This pr added parsing rules to support aliases in table value functions.
## How was this patch tested?
Added tests in `PlanParserSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17666 from maropu/SPARK-20311.
## What changes were proposed in this pull request?
So far, we do not drop all the cataloged objects after each package. Sometimes, we might hit strange test case errors because the previous test suite did not drop the cataloged/temporary objects (tables/functions/database). At least, we can first clean up the environment when completing the package of `sql/core` and `sql/hive`.
## How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17908 from gatorsmile/reset.
### What changes were proposed in this pull request?
Table comment was not getting set/unset using **ALTER TABLE SET/UNSET TBLPROPERTIES** query
eg: ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment)
when user alter the table properties and adds/updates table comment,table comment which is a field of **CatalogTable** instance is not getting updated and old table comment if exists was shown to user, inorder to handle this issue, update the comment field value in **CatalogTable** with the newly added/modified comment along with other table level properties when user executes **ALTER TABLE SET TBLPROPERTIES** query.
This pr has also taken care of unsetting the table comment when user executes query **ALTER TABLE UNSET TBLPROPERTIES** inorder to unset or remove table comment.
eg: ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment')
### How was this patch tested?
Added test cases as part of **SQLQueryTestSuite** for verifying table comment using desc formatted table query after adding/modifying table comment as part of **AlterTableSetPropertiesCommand** and unsetting the table comment using **AlterTableUnsetPropertiesCommand**.
Author: sujith71955 <sujithchacko.2010@gmail.com>
Closes#17649 from sujith71955/alter_table_comment.
## What changes were proposed in this pull request?
This change allows timestamps in parquet-based hive table to behave as a "floating time", without a timezone, as timestamps are for other file formats. If the storage timezone is the same as the session timezone, this conversion is a no-op. When data is read from a hive table, the table property is *always* respected. This allows spark to not change behavior when reading old data, but read newly written data correctly (whatever the source of the data is).
Spark inherited the original behavior from Hive, but Hive is also updating behavior to use the same scheme in HIVE-12767 / HIVE-16231.
The default for Spark remains unchanged; created tables do not include the new table property.
This will only apply to hive tables; nothing is added to parquet metadata to indicate the timezone, so data that is read or written directly from parquet files will never have any conversions applied.
## How was this patch tested?
Added a unit test which creates tables, reads and writes data, under a variety of permutations (different storage timezones, different session timezones, vectorized reading on and off).
Author: Imran Rashid <irashid@cloudera.com>
Closes#16781 from squito/SPARK-12297.
## What changes were proposed in this pull request?
* Docs are consistent (across different `unix_timestamp` variants and their internal expressions)
* typo hunting
## How was this patch tested?
local build
Author: Jacek Laskowski <jacek@japila.pl>
Closes#17801 from jaceklaskowski/unix_timestamp.
## What changes were proposed in this pull request?
Due to a likely typo, the logDebug msg printing the diff of query plans shows a diff to the initial plan, not diff to the start of batch.
## How was this patch tested?
Now the debug message prints the diff between start and end of batch.
Author: Juliusz Sompolski <julek@databricks.com>
Closes#17875 from juliuszsompolski/SPARK-20616.
## What changes were proposed in this pull request?
We allow users to specify hints (currently only "broadcast" is supported) in SQL and DataFrame. However, while SQL has a standard hint format (/*+ ... */), DataFrame doesn't have one and sometimes users are confused that they can't find how to apply a broadcast hint. This ticket adds a generic hint function on DataFrame that allows using the same hint on DataFrames as well as SQL.
As an example, after this patch, the following will apply a broadcast hint on a DataFrame using the new hint function:
```
df1.join(df2.hint("broadcast"))
```
## How was this patch tested?
Added a test case in DataFrameJoinSuite.
Author: Reynold Xin <rxin@databricks.com>
Closes#17839 from rxin/SPARK-20576.
## What changes were proposed in this pull request?
Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#17803 from srowen/SPARK-20523.
## What changes were proposed in this pull request?
A fix for the same problem was made in #17693 but ignored `JsonToStructs`. This PR uses the same fix for `JsonToStructs`.
## How was this patch tested?
Regression test
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#17826 from brkyvz/SPARK-20549.
## What changes were proposed in this pull request?
Add support for the SQL standard distinct predicate to SPARK SQL.
```
<expression> IS [NOT] DISTINCT FROM <expression>
```
## How was this patch tested?
Tested using unit tests, integration tests, manual tests.
Author: ptkool <michael.styles@shopify.com>
Closes#17764 from ptkool/is_not_distinct_from.
## What changes were proposed in this pull request?
Generate exec does not produce `null` values if the generator for the input row is empty and the generate operates in outer mode without join. This is caused by the fact that the `join=false` code path is different from the `join=true` code path, and that the `join=false` code path did deal with outer properly. This PR addresses this issue.
## How was this patch tested?
Updated `outer*` tests in `GeneratorFunctionSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17810 from hvanhovell/SPARK-20534.
## What changes were proposed in this pull request?
Currently, when the type string is invalid, it looks printing empty parentheses. This PR proposes a small improvement in an error message by removing it in the parse as below:
```scala
spark.range(1).select($"col".cast("aa"))
```
**Before**
```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa() is not supported.(line 1, pos 0)
== SQL ==
aa
^^^
```
**After**
```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa is not supported.(line 1, pos 0)
== SQL ==
aa
^^^
```
## How was this patch tested?
Unit tests in `DataTypeParserSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17784 from HyukjinKwon/SPARK-20492.
## What changes were proposed in this pull request?
This PR proposes to fill up the documentation with examples for `bitwiseOR`, `bitwiseAND`, `bitwiseXOR`. `contains`, `asc` and `desc` in `Column` API.
Also, this PR fixes minor typos in the documentation and matches some of the contents between Scala doc and Python doc.
Lastly, this PR suggests to use `spark` rather than `sc` in doc tests in `Column` for Python documentation.
## How was this patch tested?
Doc tests were added and manually tested with the commands below:
`./python/run-tests.py --module pyspark-sql`
`./python/run-tests.py --module pyspark-sql --python-executable python3`
`./dev/lint-python`
Output was checked via `make html` under `./python/docs`. The snapshots will be left on the codes with comments.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17737 from HyukjinKwon/SPARK-20442.
## What changes were proposed in this pull request?
This pr added a new rule in `Analyzer` to resolve aliases in `GROUP BY`.
The current master throws an exception if `GROUP BY` clauses have aliases in `SELECT`;
```
scala> spark.sql("select a a1, a1 + 1 as b, count(1) from t group by a1")
org.apache.spark.sql.AnalysisException: cannot resolve '`a1`' given input columns: [a]; line 1 pos 51;
'Aggregate ['a1], [a#83L AS a1#87L, ('a1 + 1) AS b#88, count(1) AS count(1)#90L]
+- SubqueryAlias t
+- Project [id#80L AS a#83L]
+- Range (0, 10, step=1, splits=Some(8))
at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:77)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:74)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
```
## How was this patch tested?
Added tests in `SQLQuerySuite` and `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17191 from maropu/SPARK-14471.
## What changes were proposed in this pull request?
Relax the requirement that a `TimeZoneAwareExpression` has to have its `timeZoneId` set to be considered resolved.
With this change, a `Cast` (which is a `TimeZoneAwareExpression`) can be considered resolved if the `(fromType, toType)` combination doesn't require time zone information.
Also de-relaxed test cases in `CastSuite` so Casts in that test suite don't get a default`timeZoneId = Option("GMT")`.
## How was this patch tested?
Ran the de-relaxed`CastSuite` and it's passing. Also ran the SQL unit tests and they're passing too.
Author: Kris Mok <kris.mok@databricks.com>
Closes#17777 from rednaxelafx/fix-catalyst-cast-timezone.
## What changes were proposed in this pull request?
change to using Jackson's `com.fasterxml.jackson.core.JsonFactory`
public JsonParser createParser(String content)
## How was this patch tested?
existing unit tests
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Eric Wasserman <ericw@sgn.com>
Closes#17693 from ewasserman/SPARK-20314.
## What changes were proposed in this pull request?
This patch adds support for customizing the spark session by injecting user-defined custom extensions. This allows a user to add custom analyzer rules/checks, optimizer rules, planning strategies or even a customized parser.
## How was this patch tested?
Unit Tests in SparkSessionExtensionSuite
Author: Sameer Agarwal <sameerag@cs.berkeley.edu>
Closes#17724 from sameeragarwal/session-extensions.
## What changes were proposed in this pull request?
This PR avoids an exception in the case where `scala.math.BigInt` has a value that does not fit into long value range (e.g. `Long.MAX_VALUE+1`). When we run the following code by using the current Spark, the following exception is thrown.
This PR keeps the value using `BigDecimal` if we detect such an overflow case by catching `ArithmeticException`.
Sample program:
```
case class BigIntWrapper(value:scala.math.BigInt)```
spark.createDataset(BigIntWrapper(scala.math.BigInt("10000000000000000002"))::Nil).show
```
Exception:
```
Error while encoding: java.lang.ArithmeticException: BigInteger out of long range
staticinvoke(class org.apache.spark.sql.types.Decimal$, DecimalType(38,0), apply, assertnotnull(assertnotnull(input[0, org.apache.spark.sql.BigIntWrapper, true])).value, true) AS value#0
java.lang.RuntimeException: Error while encoding: java.lang.ArithmeticException: BigInteger out of long range
staticinvoke(class org.apache.spark.sql.types.Decimal$, DecimalType(38,0), apply, assertnotnull(assertnotnull(input[0, org.apache.spark.sql.BigIntWrapper, true])).value, true) AS value#0
at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:290)
at org.apache.spark.sql.SparkSession$$anonfun$2.apply(SparkSession.scala:454)
at org.apache.spark.sql.SparkSession$$anonfun$2.apply(SparkSession.scala:454)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.immutable.List.foreach(List.scala:381)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.immutable.List.map(List.scala:285)
at org.apache.spark.sql.SparkSession.createDataset(SparkSession.scala:454)
at org.apache.spark.sql.Agg$$anonfun$18.apply$mcV$sp(MySuite.scala:192)
at org.apache.spark.sql.Agg$$anonfun$18.apply(MySuite.scala:192)
at org.apache.spark.sql.Agg$$anonfun$18.apply(MySuite.scala:192)
at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
at org.scalatest.Transformer.apply(Transformer.scala:22)
at org.scalatest.Transformer.apply(Transformer.scala:20)
at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
...
Caused by: java.lang.ArithmeticException: BigInteger out of long range
at java.math.BigInteger.longValueExact(BigInteger.java:4531)
at org.apache.spark.sql.types.Decimal.set(Decimal.scala:140)
at org.apache.spark.sql.types.Decimal$.apply(Decimal.scala:434)
at org.apache.spark.sql.types.Decimal.apply(Decimal.scala)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:287)
... 59 more
```
## How was this patch tested?
Add new test suite into `DecimalSuite`
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#17684 from kiszk/SPARK-20341.
## What changes were proposed in this pull request?
If a partitionSpec is supposed to not contain optional values, a ParseException should be thrown, and not nulls returned.
The nulls can later cause NullPointerExceptions in places not expecting them.
## How was this patch tested?
A query like "SHOW PARTITIONS tbl PARTITION(col1='val1', col2)" used to throw a NullPointerException.
Now it throws a ParseException.
Author: Juliusz Sompolski <julek@databricks.com>
Closes#17707 from juliuszsompolski/SPARK-20412.
## What changes were proposed in this pull request?
It is often useful to be able to track changes to the `ExternalCatalog`. This PR makes the `ExternalCatalog` emit events when a catalog object is changed. Events are fired before and after the change.
The following events are fired per object:
- Database
- CreateDatabasePreEvent: event fired before the database is created.
- CreateDatabaseEvent: event fired after the database has been created.
- DropDatabasePreEvent: event fired before the database is dropped.
- DropDatabaseEvent: event fired after the database has been dropped.
- Table
- CreateTablePreEvent: event fired before the table is created.
- CreateTableEvent: event fired after the table has been created.
- RenameTablePreEvent: event fired before the table is renamed.
- RenameTableEvent: event fired after the table has been renamed.
- DropTablePreEvent: event fired before the table is dropped.
- DropTableEvent: event fired after the table has been dropped.
- Function
- CreateFunctionPreEvent: event fired before the function is created.
- CreateFunctionEvent: event fired after the function has been created.
- RenameFunctionPreEvent: event fired before the function is renamed.
- RenameFunctionEvent: event fired after the function has been renamed.
- DropFunctionPreEvent: event fired before the function is dropped.
- DropFunctionPreEvent: event fired after the function has been dropped.
The current events currently only contain the names of the object modified. We add more events, and more details at a later point.
A user can monitor changes to the external catalog by adding a listener to the Spark listener bus checking for `ExternalCatalogEvent`s using the `SparkListener.onOtherEvent` hook. A more direct approach is add listener directly to the `ExternalCatalog`.
## How was this patch tested?
Added the `ExternalCatalogEventSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17710 from hvanhovell/SPARK-20420.
## What changes were proposed in this pull request?
A cast expression with a resolved time zone is not equal to a cast expression without a resolved time zone. The `ResolveAggregateFunction` assumed that these expression were the same, and would fail to resolve `HAVING` clauses which contain a `Cast` expression.
This is in essence caused by the fact that a `TimeZoneAwareExpression` can be resolved without a set time zone. This PR fixes this, and makes a `TimeZoneAwareExpression` unresolved as long as it has no TimeZone set.
## How was this patch tested?
Added a regression test to the `SQLQueryTestSuite.having` file.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17641 from hvanhovell/SPARK-20329.
## What changes were proposed in this pull request?
Address a follow up in [comment](https://github.com/apache/spark/pull/16954#discussion_r105718880)
Currently subqueries with correlated predicates containing aggregate expression having mixture of outer references and local references generate a codegen error like following :
```SQL
SELECT t1a
FROM t1
GROUP BY 1
HAVING EXISTS (SELECT 1
FROM t2
WHERE t2a < min(t1a + t2a));
```
Exception snippet.
```
Cannot evaluate expression: min((input[0, int, false] + input[4, int, false]))
at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.doGenCode(Expression.scala:226)
at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:87)
at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:106)
at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:103)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:103)
```
After this PR, a better error message is issued.
```
org.apache.spark.sql.AnalysisException
Error in query: Found an aggregate expression in a correlated
predicate that has both outer and local references, which is not supported yet.
Aggregate expression: min((t1.`t1a` + t2.`t2a`)),
Outer references: t1.`t1a`,
Local references: t2.`t2a`.;
```
## How was this patch tested?
Added tests in SQLQueryTestSuite.
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#17636 from dilipbiswal/subquery_followup1.
## What changes were proposed in this pull request?
It's illegal to have aggregate function in GROUP BY, and we should fail at analysis phase, if this happens.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17704 from cloud-fan/minor.
### What changes were proposed in this pull request?
Database and Table names conform the Hive standard ("[a-zA-z_0-9]+"), i.e. if this name only contains characters, numbers, and _.
When calling `toLowerCase` on the names, we should add `Locale.ROOT` to the `toLowerCase`for avoiding inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").
### How was this patch tested?
Added a test case
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17655 from gatorsmile/locale.
## What changes were proposed in this pull request?
Also went through the same file to ensure other string concatenation are correct.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17691 from zsxwing/fix-error-message.
## What changes were proposed in this pull request?
Apply Complementation Laws during boolean expression simplification.
## How was this patch tested?
Tested using unit tests, integration tests, and manual tests.
Author: ptkool <michael.styles@shopify.com>
Author: Michael Styles <michael.styles@shopify.com>
Closes#17650 from ptkool/apply_complementation_laws.
Avoid necessary execution that can lead to NPE in EliminateOuterJoin and add test in DataFrameSuite to confirm NPE is no longer thrown
## What changes were proposed in this pull request?
Change leftHasNonNullPredicate and rightHasNonNullPredicate to lazy so they are only executed when needed.
## How was this patch tested?
Added test in DataFrameSuite that failed before this fix and now succeeds. Note that a test in catalyst project would be better but i am unsure how to do this.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Koert Kuipers <koert@tresata.com>
Closes#17660 from koertkuipers/feat-catch-npe-in-eliminate-outer-join.
## What changes were proposed in this pull request?
If a plan has multi-level successive joins, e.g.:
```
Join
/ \
Union t5
/ \
Join t4
/ \
Join t3
/ \
t1 t2
```
Currently we fail to reorder the inside joins, i.e. t1, t2, t3.
In join reorder, we use `OrderedJoin` to indicate a join has been ordered, such that when transforming down the plan, these joins don't need to be rerodered again.
But there's a problem in the definition of `OrderedJoin`:
The real join node is a parameter, but not a child. This breaks the transform procedure because `mapChildren` applies transform function on parameters which should be children.
In this patch, we change `OrderedJoin` to a class having the same structure as a join node.
## How was this patch tested?
Add a corresponding test case.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17668 from wzhfy/recursiveReorder.
## What changes were proposed in this pull request?
fix typo
## How was this patch tested?
manual
Author: Felix Cheung <felixcheung_m@hotmail.com>
Closes#17663 from felixcheung/likedoctypo.
## What changes were proposed in this pull request?
Replace non-existent `repartitionBy` with `distribute` in `CollapseRepartitionSuite`.
## How was this patch tested?
local build and `catalyst/testOnly *CollapseRepartitionSuite`
Author: Jacek Laskowski <jacek@japila.pl>
Closes#17657 from jaceklaskowski/CollapseRepartitionSuite.
## What changes were proposed in this pull request?
This patch fixes a bug in the way LIKE patterns are translated to Java regexes. The bug causes any character following an escaped backslash to be escaped, i.e. there is double-escaping.
A concrete example is the following pattern:`'%\\%'`. The expected Java regex that this pattern should correspond to (according to the behavior described below) is `'.*\\.*'`, however the current situation leads to `'.*\\%'` instead.
---
Update: in light of the discussion that ensued, we should explicitly define the expected behaviour of LIKE expressions, especially in certain edge cases. With the help of gatorsmile, we put together a list of different RDBMS and their variations wrt to certain standard features.
| RDBMS\Features | Wildcards | Default escape [1] | Case sensitivity |
| --- | --- | --- | --- |
| [MS SQL Server](https://msdn.microsoft.com/en-us/library/ms179859.aspx) | _, %, [], [^] | none | no |
| [Oracle](https://docs.oracle.com/cd/B12037_01/server.101/b10759/conditions016.htm) | _, % | none | yes |
| [DB2 z/OS](http://www.ibm.com/support/knowledgecenter/SSEPEK_11.0.0/sqlref/src/tpc/db2z_likepredicate.html) | _, % | none | yes |
| [MySQL](http://dev.mysql.com/doc/refman/5.7/en/string-comparison-functions.html) | _, % | none | no |
| [PostreSQL](https://www.postgresql.org/docs/9.0/static/functions-matching.html) | _, % | \ | yes |
| [Hive](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF) | _, % | none | yes |
| Current Spark | _, % | \ | yes |
[1] Default escape character: most systems do not have a default escape character, instead the user can specify one by calling a like expression with an escape argument [A] LIKE [B] ESCAPE [C]. This syntax is currently not supported by Spark, however I would volunteer to implement this feature in a separate ticket.
The specifications are often quite terse and certain scenarios are undocumented, so here is a list of scenarios that I am uncertain about and would appreciate any input. Specifically I am looking for feedback on whether or not Spark's current behavior should be changed.
1. [x] Ending a pattern with the escape sequence, e.g. `like 'a\'`.
PostreSQL gives an error: 'LIKE pattern must not end with escape character', which I personally find logical. Currently, Spark allows "non-terminated" escapes and simply ignores them as part of the pattern.
According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), ending a pattern in an escape character is invalid.
_Proposed new behaviour in Spark: throw AnalysisException_
2. [x] Empty input, e.g. `'' like ''`
Postgres and DB2 will match empty input only if the pattern is empty as well, any other combination of empty input will not match. Spark currently follows this rule.
3. [x] Escape before a non-special character, e.g. `'a' like '\a'`.
Escaping a non-wildcard character is not really documented but PostgreSQL just treats it verbatim, which I also find the least surprising behavior. Spark does the same.
According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), it is invalid to follow an escape character with anything other than an escape character, an underscore or a percent sign.
_Proposed new behaviour in Spark: throw AnalysisException_
The current specification is also described in the operator's source code in this patch.
## How was this patch tested?
Extra case in regex unit tests.
Author: Jakob Odersky <jakob@odersky.com>
This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@databricks.com>
Closes#15398 from jodersky/SPARK-17647.
### What changes were proposed in this pull request?
The session catalog caches some persistent functions in the `FunctionRegistry`, so there can be duplicates. Our Catalog API `listFunctions` does not handle it.
It would be better if `SessionCatalog` API can de-duplciate the records, instead of doing it by each API caller. In `FunctionRegistry`, our functions are identified by the unquoted string. Thus, this PR is try to parse it using our parser interface and then de-duplicate the names.
### How was this patch tested?
Added test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17646 from gatorsmile/showFunctions.
## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/17398 we introduced `UnresolvedMapObjects` as a placeholder of `MapObjects`. Unfortunately `UnresolvedMapObjects` is not serializable as its `function` may reference Scala `Type` which is not serializable.
Ideally this is fine, as we will never serialize and send unresolved expressions to executors. However users may accidentally do this, e.g. mistakenly reference an encoder instance when implementing `Aggregator`, we should fix it so that it's just a performance issue(more network traffic) and should not fail the query.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17639 from cloud-fan/minor.
## What changes were proposed in this pull request?
Currently when estimating predicates like col > literal or col = literal, we will update min or max in column stats based on literal value. However, literal value is of Catalyst type (internal type), while min/max is of external type. Then for the next predicate, we again need to do type conversion to compare and update column stats. This is awkward and causes many unnecessary conversions in estimation.
To solve this, we use Catalyst type for min/max in `ColumnStat`. Note that the persistent format in metastore is still of external type, so there's no inconsistency for statistics in metastore.
This pr also fixes a bug for boolean type in `IN` condition.
## How was this patch tested?
The changes for ColumnStat are covered by existing tests.
For bug fix, a new test for boolean type in IN condition is added
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17630 from wzhfy/refactorColumnStat.
## What changes were proposed in this pull request?
AssertNotNull's toString/simpleString dumps the entire walkedTypePath. walkedTypePath is used for error message reporting and shouldn't be part of the output.
## How was this patch tested?
Manually tested.
Author: Reynold Xin <rxin@databricks.com>
Closes#17616 from rxin/SPARK-20304.
### What changes were proposed in this pull request?
Session catalog API `createTempFunction` is being used by Hive build-in functions, persistent functions, and temporary functions. Thus, the name is confusing. This PR is to rename it by `registerFunction`. Also we can move construction of `FunctionBuilder` and `ExpressionInfo` into the new `registerFunction`, instead of duplicating the logics everywhere.
In the next PRs, the remaining Function-related APIs also need cleanups.
### How was this patch tested?
Existing test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17615 from gatorsmile/cleanupCreateTempFunction.
## What changes were proposed in this pull request?
This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable.
There are several problems with it:
- It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?".
- > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up.
(see joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627))
To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above.
There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013
Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings.
## How was this patch tested?
Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`.
This was tested via manually adding `time.time()` as below:
```diff
profiles_and_goals = build_profiles + sbt_goals
print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
" ".join(profiles_and_goals))
+ import time
+ st = time.time()
exec_sbt(profiles_and_goals)
+ print("Elapsed :[%s]" % str(time.time() - st))
```
produces
```
...
========================================================================
Building Unidoc API Documentation
========================================================================
...
[info] Main Java API documentation successful.
...
Elapsed :[94.8746569157]
...
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17477 from HyukjinKwon/SPARK-18692.
## What changes were proposed in this pull request?
Update count distinct error message for streaming datasets/dataframes to match current behavior. These aggregations are not yet supported, regardless of whether the dataset/dataframe is aggregated.
Author: jtoka <jason.tokayer@gmail.com>
Closes#17609 from jtoka/master.
## What changes were proposed in this pull request?
When we perform a cast expression and the from and to types are structurally the same (having the same structure but different field names), we should be able to skip the actual cast.
## How was this patch tested?
Added unit tests for the newly introduced functions.
Author: Reynold Xin <rxin@databricks.com>
Closes#17614 from rxin/SPARK-20302.
## What changes were proposed in this pull request?
`NaNvl(float value, null)` will be converted into `NaNvl(float value, Cast(null, DoubleType))` and finally `NaNvl(Cast(float value, DoubleType), Cast(null, DoubleType))`.
This will cause mismatching in the output type when the input type is float.
By adding extra rule in TypeCoercion can resolve this issue.
## How was this patch tested?
unite tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: DB Tsai <dbt@netflix.com>
Closes#17606 from dbtsai/fixNaNvl.
## What changes were proposed in this pull request?
Dataset typed API currently uses NewInstance to box primitive types (i.e. calling the constructor). Instead, it'd be slightly more idiomatic in Java to use PrimitiveType.valueOf, which can be invoked using StaticInvoke expression.
## How was this patch tested?
The change should be covered by existing tests for Dataset encoders.
Author: Reynold Xin <rxin@databricks.com>
Closes#17604 from rxin/SPARK-20289.
## What changes were proposed in this pull request?
Similar to `ListQuery`, `Exists` should not be evaluated in `Join` operator too.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#17491 from viirya/dont-push-exists-to-join.
## What changes were proposed in this pull request?
This is a regression caused by SPARK-19716.
Before SPARK-19716, we will cast an array field to the expected array type. However, after SPARK-19716, the cast is removed, but we forgot to push the cast to the element level.
## How was this patch tested?
new regression tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17587 from cloud-fan/array.
## What changes were proposed in this pull request?
Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").
The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#17527 from srowen/SPARK-20156.
## What changes were proposed in this pull request?
```
sql("SELECT t1.b, rand(0) as r FROM cachedData, cachedData t1 GROUP BY t1.b having r > 0.5").show()
```
We will get the following error:
```
Job aborted due to stage failure: Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 8, localhost, executor driver): java.lang.NullPointerException
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source)
at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$1.apply(BroadcastNestedLoopJoinExec.scala:87)
at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$1.apply(BroadcastNestedLoopJoinExec.scala:87)
at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:463)
```
Filters could be pushed down to the join conditions by the optimizer rule `PushPredicateThroughJoin`. However, Analyzer [blocks users to add non-deterministics conditions](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala#L386-L395) (For details, see the PR https://github.com/apache/spark/pull/7535).
We should not push down non-deterministic conditions; otherwise, we need to explicitly initialize the non-deterministic expressions. This PR is to simply block it.
### How was this patch tested?
Added a test case
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17585 from gatorsmile/joinRandCondition.
## What changes were proposed in this pull request?
This PR proposes to add `IGNORE NULLS` keyword in `first`/`last` in Spark's parser likewise http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions057.htm. This simply maps the keywords to existing `ignoreNullsExpr`.
**Before**
```scala
scala> sql("select first('a' IGNORE NULLS)").show()
```
```
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input 'NULLS' expecting {')', ','}(line 1, pos 24)
== SQL ==
select first('a' IGNORE NULLS)
------------------------^^^
at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:210)
at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:112)
at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:46)
at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:66)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:622)
... 48 elided
```
**After**
```scala
scala> sql("select first('a' IGNORE NULLS)").show()
```
```
+--------------+
|first(a, true)|
+--------------+
| a|
+--------------+
```
## How was this patch tested?
Unit tests in `ExpressionParserSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17566 from HyukjinKwon/SPARK-19518.
## What changes were proposed in this pull request?
Like `Expression`, `QueryPlan` should also have a `semanticHash` method, then we can put plans to a hash map and look it up fast. This PR refactors `QueryPlan` to follow `Expression` and put all the normalization logic in `QueryPlan.canonicalized`, so that it's very natural to implement `semanticHash`.
follow-up: improve `CacheManager` to leverage this `semanticHash` and speed up plan lookup, instead of iterating all cached plans.
## How was this patch tested?
existing tests. Note that we don't need to test the `semanticHash` method, once the existing tests prove `sameResult` is correct, we are good.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17541 from cloud-fan/plan-semantic.
## What changes were proposed in this pull request?
AssertNotNull currently throws RuntimeException. It should throw NullPointerException, which is more specific.
## How was this patch tested?
N/A
Author: Reynold Xin <rxin@databricks.com>
Closes#17573 from rxin/SPARK-20262.
## What changes were proposed in this pull request?
Similar to `Project`, when `Aggregate` has non-deterministic expressions, we should not push predicate down through it, as it will change the number of input rows and thus change the evaluation result of non-deterministic expressions in `Aggregate`.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17562 from cloud-fan/filter.
## What changes were proposed in this pull request?
Currently `LogicalRelation` has a `expectedOutputAttributes` parameter, which makes it hard to reason about what the actual output is. Like other leaf nodes, `LogicalRelation` should also take `output` as a parameter, to simplify the logic
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17552 from cloud-fan/minor.
## What changes were proposed in this pull request?
This is a tiny addendum to SPARK-19495 to remove the private visibility for copy, which is the only package private method in the entire file.
## How was this patch tested?
N/A - no semantic change.
Author: Reynold Xin <rxin@databricks.com>
Closes#17555 from rxin/SPARK-19495-2.
## What changes were proposed in this pull request?
This commit moves star schema code from ```join.scala``` to ```StarSchemaDetection.scala```. It also applies some minor fixes in ```StarJoinReorderSuite.scala```.
## How was this patch tested?
Run existing ```StarJoinReorderSuite.scala```.
Author: Ioana Delaney <ioanamdelaney@gmail.com>
Closes#17544 from ioana-delaney/starSchemaCBOv2.
## What changes were proposed in this pull request?
Make sure SESSION_LOCAL_TIMEZONE reflects the change in JVM's default timezone setting. Currently several timezone related tests fail as the change to default timezone is not picked up by SQLConf.
## How was this patch tested?
Added an unit test in ConfigEntrySuite
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#17537 from dilipbiswal/timezone_debug.
## What changes were proposed in this pull request?
Previously when we construct deserializer expression for array type, we will first cast the corresponding field to expected array type and then apply `MapObjects`.
However, by doing that, we lose the opportunity to do by-name resolution for struct type inside array type. In this PR, I introduce a `UnresolvedMapObjects` to hold the lambda function and the input array expression. Then during analysis, after the input array expression is resolved, we get the actual array element type and apply by-name resolution. Then we don't need to add `Cast` for array type when constructing the deserializer expression, as the element type is determined later at analyzer.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17398 from cloud-fan/dataset.
## What changes were proposed in this pull request?
This is a follow-up of https://github.com/apache/spark/pull/17285 .
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17521 from cloud-fan/conf.
### What changes were proposed in this pull request?
Observed by felixcheung , in `SparkSession`.`Catalog` APIs, we have different conventions/rules for table/function identifiers/names. Most APIs accept the qualified name (i.e., `databaseName`.`tableName` or `databaseName`.`functionName`). However, the following five APIs do not accept it.
- def listColumns(tableName: String): Dataset[Column]
- def getTable(tableName: String): Table
- def getFunction(functionName: String): Function
- def tableExists(tableName: String): Boolean
- def functionExists(functionName: String): Boolean
To make them consistent with the other Catalog APIs, this PR does the changes, updates the function/API comments and adds the `params` to clarify the inputs we allow.
### How was this patch tested?
Added the test cases .
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17518 from gatorsmile/tableIdentifier.
### What changes were proposed in this pull request?
This PR is to unify and clean up the outputs of `DESC EXTENDED/FORMATTED` and `SHOW TABLE EXTENDED` by moving the logics into the Catalog interface. The output formats are improved. We also add the missing attributes. It impacts the DDL commands like `SHOW TABLE EXTENDED`, `DESC EXTENDED` and `DESC FORMATTED`.
In addition, by following what we did in Dataset API `printSchema`, we can use `treeString` to show the schema in the more readable way.
Below is the current way:
```
Schema: STRUCT<`a`: STRING (nullable = true), `b`: INT (nullable = true), `c`: STRING (nullable = true), `d`: STRING (nullable = true)>
```
After the change, it should look like
```
Schema: root
|-- a: string (nullable = true)
|-- b: integer (nullable = true)
|-- c: string (nullable = true)
|-- d: string (nullable = true)
```
### How was this patch tested?
`describe.sql` and `show-tables.sql`
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17394 from gatorsmile/descFollowUp.
## What changes were proposed in this pull request?
**Description** from JIRA
The TimestampType in Spark SQL is of microsecond precision. Ideally, we should convert Spark SQL timestamp values into Parquet TIMESTAMP_MICROS. But unfortunately parquet-mr hasn't supported it yet.
For the read path, we should be able to read TIMESTAMP_MILLIS Parquet values and pad a 0 microsecond part to read values.
For the write path, currently we are writing timestamps as INT96, similar to Impala and Hive. One alternative is that, we can have a separate SQL option to let users be able to write Spark SQL timestamp values as TIMESTAMP_MILLIS. Of course, in this way the microsecond part will be truncated.
## How was this patch tested?
Added new tests in ParquetQuerySuite and ParquetIOSuite
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#15332 from dilipbiswal/parquet-time-millis.
## What changes were proposed in this pull request?
In SQL queries, we also see predicate expressions involving two columns such as "column-1 (op) column-2" where column-1 and column-2 belong to same table. Note that, if column-1 and column-2 belong to different tables, then it is a join operator's work, NOT a filter operator's work.
This PR estimates filter selectivity on two columns of same table. For example, multiple tpc-h queries have this predicate "WHERE l_commitdate < l_receiptdate"
## How was this patch tested?
We added 6 new test cases to test various logical predicates involving two columns of same table.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Ron Hu <ron.hu@huawei.com>
Author: U-CHINA\r00754707 <r00754707@R00754707-SC04.china.huawei.com>
Closes#17415 from ron8hu/filterTwoColumns.
## What changes were proposed in this pull request?
Range in SQL should be case insensitive
## How was this patch tested?
unit test
Author: samelamin <hussam.elamin@gmail.com>
Author: samelamin <sam_elamin@discovery.com>
Closes#17487 from samelamin/SPARK-20145.
## What changes were proposed in this pull request?
This patch implements `listPartitionsByFilter()` for `InMemoryCatalog` and thus resolves an outstanding TODO causing the `PruneFileSourcePartitions` optimizer rule not to apply when "spark.sql.catalogImplementation" is set to "in-memory" (which is the default).
The change is straightforward: it extracts the code for further filtering of the list of partitions returned by the metastore's `getPartitionsByFilter()` out from `HiveExternalCatalog` into `ExternalCatalogUtils` and calls this new function from `InMemoryCatalog` on the whole list of partitions.
Now that this method is implemented we can always pass the `CatalogTable` to the `DataSource` in `FindDataSourceTable`, so that the latter is resolved to a relation with a `CatalogFileIndex`, which is what the `PruneFileSourcePartitions` rule matches for.
## How was this patch tested?
Ran existing tests and added new test for `listPartitionsByFilter` in `ExternalCatalogSuite`, which is subclassed by both `InMemoryCatalogSuite` and `HiveExternalCatalogSuite`.
Author: Adrian Ionescu <adrian@databricks.com>
Closes#17510 from adrian-ionescu/InMemoryCatalog.
## What changes were proposed in this pull request?
This PR proposes to use `XXX` format instead of `ZZ`. `ZZ` seems a `FastDateFormat` specific.
`ZZ` supports "ISO 8601 extended format time zones" but it seems `FastDateFormat` specific option.
I misunderstood this is compatible format with `SimpleDateFormat` when this change is introduced.
Please see [SimpleDateFormat documentation]( https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html#iso8601timezone) and [FastDateFormat documentation](https://commons.apache.org/proper/commons-lang/apidocs/org/apache/commons/lang3/time/FastDateFormat.html).
It seems we better replace `ZZ` to `XXX` because they look using the same strategy - [FastDateParser.java#L930](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L930)), [FastDateParser.java#L932-L951 ](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L932-L951)) and [FastDateParser.java#L596-L601](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L596-L601)).
I also checked the codes and manually debugged it for sure. It seems both cases use the same pattern `( Z|(?:[+-]\\d{2}(?::)\\d{2}))`.
_Note that this should be rather a fix about documentation and not the behaviour change because `ZZ` seems invalid date format in `SimpleDateFormat` as documented in `DataFrameReader` and etc, and both `ZZ` and `XXX` look identically working with `FastDateFormat`_
Current documentation is as below:
```
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that
* indicates a timestamp format. Custom date formats follow the formats at
* `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
```
## How was this patch tested?
Existing tests should cover this. Also, manually tested as below (BTW, I don't think these are worth being added as tests within Spark):
**Parse**
```scala
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")
res4: java.util.Date = Tue Mar 21 20:00:00 KST 2017
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z")
res10: java.util.Date = Tue Mar 21 09:00:00 KST 2017
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00")
java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000-11:00"
at java.text.DateFormat.parse(DateFormat.java:366)
... 48 elided
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z")
java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000Z"
at java.text.DateFormat.parse(DateFormat.java:366)
... 48 elided
```
```scala
scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")
res7: java.util.Date = Tue Mar 21 20:00:00 KST 2017
scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z")
res1: java.util.Date = Tue Mar 21 09:00:00 KST 2017
scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00")
res8: java.util.Date = Tue Mar 21 20:00:00 KST 2017
scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z")
res2: java.util.Date = Tue Mar 21 09:00:00 KST 2017
```
**Format**
```scala
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00"))
res6: String = 2017-03-21T20:00:00.000+09:00
```
```scala
scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ")
fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSZZ,ko_KR,Asia/Seoul]
scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00"))
res1: String = 2017-03-21T20:00:00.000+09:00
scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX")
fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSXXX,ko_KR,Asia/Seoul]
scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00"))
res2: String = 2017-03-21T20:00:00.000+09:00
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17489 from HyukjinKwon/SPARK-20166.
## What changes were proposed in this pull request?
Currently, `DataType.fromJson` throws `scala.MatchError` or `java.util.NoSuchElementException` in some cases when the JSON input is invalid as below:
```scala
DataType.fromJson(""""abcd"""")
```
```
java.util.NoSuchElementException: key not found: abcd
at ...
```
```scala
DataType.fromJson("""{"abcd":"a"}""")
```
```
scala.MatchError: JObject(List((abcd,JString(a)))) (of class org.json4s.JsonAST$JObject)
at ...
```
```scala
DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""")
```
```
scala.MatchError: JObject(List((a,JInt(123)))) (of class org.json4s.JsonAST$JObject)
at ...
```
After this PR,
```scala
DataType.fromJson(""""abcd"""")
```
```
java.lang.IllegalArgumentException: Failed to convert the JSON string 'abcd' to a data type.
at ...
```
```scala
DataType.fromJson("""{"abcd":"a"}""")
```
```
java.lang.IllegalArgumentException: Failed to convert the JSON string '{"abcd":"a"}' to a data type.
at ...
```
```scala
DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""")
at ...
```
```
java.lang.IllegalArgumentException: Failed to convert the JSON string '{"a":123}' to a field.
```
## How was this patch tested?
Unit test added in `DataTypeSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17468 from HyukjinKwon/fromjson_exception.
## What changes were proposed in this pull request?
`BroadcastHint` should use child's statistics and set `isBroadcastable` to true.
## How was this patch tested?
Added a new stats estimation test for `BroadcastHint`.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17504 from wzhfy/broadcastHintEstimation.
## What changes were proposed in this pull request?
The query plan in an `AnalysisException` may be `null` when an `AnalysisException` object is serialized and then deserialized, since `plan` is marked `transient`. Or when someone throws an `AnalysisException` with a null query plan (which should not happen).
`def getMessage` is not tolerant of this and throws a `NullPointerException`, leading to loss of information about the original exception.
The fix is to add a `null` check in `getMessage`.
## How was this patch tested?
- Unit test
Author: Kunal Khamar <kkhamar@outlook.com>
Closes#17486 from kunalkhamar/spark-20164.
## What changes were proposed in this pull request?
Instead of iterating all expressions that can return null for null inputs, we can just check `NullIntolerant`.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17450 from cloud-fan/null.
…adoc
## What changes were proposed in this pull request?
Use recommended values for row boundaries in Window's scaladoc, i.e. `Window.unboundedPreceding`, `Window.unboundedFollowing`, and `Window.currentRow` (that were introduced in 2.1.0).
## How was this patch tested?
Local build
Author: Jacek Laskowski <jacek@japila.pl>
Closes#17417 from jaceklaskowski/window-expression-scaladoc.
## What changes were proposed in this pull request?
Commit 6c70a38 broke the build for scala 2.10. The commit uses some reflections which are not available in Scala 2.10. This PR fixes them.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#17473 from ueshin/issues/SPARK-19088.
### What changes were proposed in this pull request?
`FalseLiteral` and `TrueLiteral` should have been eliminated by optimizer rule `BooleanSimplification`, but null literals might be added by optimizer rule `NullPropagation`. For safety, our filter estimation should handle all the eligible literal cases.
Our optimizer rule BooleanSimplification is unable to remove the null literal in many cases. For example, `a < 0 or null`. Thus, we need to handle null literal in filter estimation.
`Not` can be pushed down below `And` and `Or`. Then, we could see two consecutive `Not`, which need to be collapsed into one. Because of the limited expression support for filter estimation, we just need to handle the case `Not(null)` for avoiding incorrect error due to the boolean operation on null. For details, see below matrix.
```
not NULL = NULL
NULL or false = NULL
NULL or true = true
NULL or NULL = NULL
NULL and false = false
NULL and true = NULL
NULL and NULL = NULL
```
### How was this patch tested?
Added the test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17446 from gatorsmile/constantFilterEstimation.
## What changes were proposed in this pull request?
This pr added `StructType.fromDDL` to convert a DDL format string into `StructType` for defining schemas in `functions.from_json`.
## How was this patch tested?
Added tests in `JsonFunctionsSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17406 from maropu/SPARK-20009.
## What changes were proposed in this pull request?
When we build the deserializer expression for map type, we will use `StaticInvoke` to call `ArrayBasedMapData.toScalaMap`, and declare the return type as `scala.collection.immutable.Map`. If the map is inside an Option, we will wrap this `StaticInvoke` with `WrapOption`, which requires the input to be `scala.collect.Map`. Ideally this should be fine, as `scala.collection.immutable.Map` extends `scala.collect.Map`, but our `ObjectType` is too strict about this, this PR fixes it.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17454 from cloud-fan/map.
## What changes were proposed in this pull request?
Join reorder algorithm should keep exactly the same order of output attributes in the top project.
For example, if user want to select a, b, c, after reordering, we should output a, b, c in the same order as specified by user, instead of b, a, c or other orders.
## How was this patch tested?
A new test case is added in `JoinReorderSuite`.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17453 from wzhfy/keepOrderInProject.
## What changes were proposed in this pull request?
TPCDS q45 fails becuase:
`ReorderJoin` collects all predicates and try to put them into join condition when creating ordered join. If a predicate with an IN subquery (`ListQuery`) is in a join condition instead of a filter condition, `RewritePredicateSubquery.rewriteExistentialExpr` would fail to convert the subquery to an `ExistenceJoin`, and thus result in error.
We should prevent push down of IN subquery to Join operator.
## How was this patch tested?
Add a new test case in `FilterPushdownSuite`.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17428 from wzhfy/noSubqueryInJoinCond.
## What changes were proposed in this pull request?
The current SessionState initialization code path is quite complex. A part of the creation is done in the SessionState companion objects, a part of the creation is one inside the SessionState class, and a part is done by passing functions.
This PR refactors this code path, and consolidates SessionState initialization into a builder class. This SessionState will not do any initialization and just becomes a place holder for the various Spark SQL internals. This also lays the ground work for two future improvements:
1. This provides us with a start for removing the `HiveSessionState`. Removing the `HiveSessionState` would also require us to move resource loading into a separate class, and to (re)move metadata hive.
2. This makes it easier to customize the Spark Session. Currently you will need to create a custom version of the builder. I have added hooks to facilitate this. A future step will be to create a semi stable API on top of this.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17433 from hvanhovell/SPARK-20100.
## What changes were proposed in this pull request?
In current stage, we don't have advanced statistics such as sketches or histograms. As a result, some operator can't estimate `nullCount` accurately. E.g. left outer join estimation does not accurately update `nullCount` currently. So for `IsNull` and `IsNotNull` predicates, we only estimate them when the child is a leaf node, whose `nullCount` is accurate.
## How was this patch tested?
A new test case is added in `FilterEstimationSuite`.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17438 from wzhfy/nullEstimation.
## What changes were proposed in this pull request?
The `CollapseWindow` is currently to aggressive when collapsing adjacent windows. It also collapses windows in the which the parent produces a column that is consumed by the child; this creates an invalid window which will fail at runtime.
This PR fixes this by adding a check for dependent adjacent windows to the `CollapseWindow` rule.
## How was this patch tested?
Added a new test case to `CollapseWindowSuite`
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17432 from hvanhovell/SPARK-20086.
## What changes were proposed in this pull request?
The `FailureSafeParser` is only used in sql core, it doesn't make sense to put it in catalyst module.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17408 from cloud-fan/minor.
## What changes were proposed in this pull request?
Constraint propagation can be computation expensive and block the driver execution for long time. For example, the below benchmark needs 30mins.
Compared with previous PRs #16998, #16785, this is a much simpler option: add a flag to disable constraint propagation.
### Benchmark
Run the following codes locally.
import org.apache.spark.ml.{Pipeline, PipelineStage}
import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer, VectorAssembler}
import org.apache.spark.sql.internal.SQLConf
spark.conf.set(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key, false)
val df = (1 to 40).foldLeft(Seq((1, "foo"), (2, "bar"), (3, "baz")).toDF("id", "x0"))((df, i) => df.withColumn(s"x$i", $"x0"))
val indexers = df.columns.tail.map(c => new StringIndexer()
.setInputCol(c)
.setOutputCol(s"${c}_indexed")
.setHandleInvalid("skip"))
val encoders = indexers.map(indexer => new OneHotEncoder()
.setInputCol(indexer.getOutputCol)
.setOutputCol(s"${indexer.getOutputCol}_encoded")
.setDropLast(true))
val stages: Array[PipelineStage] = indexers ++ encoders
val pipeline = new Pipeline().setStages(stages)
val startTime = System.nanoTime
pipeline.fit(df).transform(df).show
val runningTime = System.nanoTime - startTime
Before this patch: 1786001 ms ~= 30 mins
After this patch: 26392 ms = less than half of a minute
Related PRs: #16998, #16785.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#17186 from viirya/add-flag-disable-constraint-propagation.
## What changes were proposed in this pull request?
Fix for typo in Analyzer
## How was this patch tested?
local build
Author: Jacek Laskowski <jacek@japila.pl>
Closes#17409 from jaceklaskowski/analyzer-typo.
## What changes were proposed in this pull request?
Since the state is tied a "group" in the "mapGroupsWithState" operations, its better to call the state "GroupState" instead of a key. This would make it more general if you extends this operation to RelationGroupedDataset and python APIs.
## How was this patch tested?
Existing unit tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#17385 from tdas/SPARK-20057.
## What changes were proposed in this pull request?
Currently, when we perform count with timestamp types, it prints the internal representation as the column name as below:
```scala
Seq(new java.sql.Timestamp(1)).toDF("a").groupBy("a").pivot("a").count().show()
```
```
+--------------------+----+
| a|1000|
+--------------------+----+
|1969-12-31 16:00:...| 1|
+--------------------+----+
```
This PR proposes to use external Scala value instead of the internal representation in the column names as below:
```
+--------------------+-----------------------+
| a|1969-12-31 16:00:00.001|
+--------------------+-----------------------+
|1969-12-31 16:00:...| 1|
+--------------------+-----------------------+
```
## How was this patch tested?
Unit test in `DataFramePivotSuite` and manual tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17348 from HyukjinKwon/SPARK-20018.
## What changes were proposed in this pull request?
This PR proposes to make `mode` options in both CSV and JSON to use `cass object` and fix some related comments related previous fix.
Also, this PR modifies some tests related parse modes.
## How was this patch tested?
Modified unit tests in both `CSVSuite.scala` and `JsonSuite.scala`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17377 from HyukjinKwon/SPARK-19949.
## What changes were proposed in this pull request?
Adding event time based timeout. The user sets the timeout timestamp directly using `KeyedState.setTimeoutTimestamp`. The keys times out when the watermark crosses the timeout timestamp.
## How was this patch tested?
Unit tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#17361 from tdas/SPARK-20030.
## What changes were proposed in this pull request?
Change the nullability of function `StringToMap` from `false` to `true`.
Author: zhaorongsheng <334362872@qq.com>
Closes#17350 from zhaorongsheng/bug-fix_strToMap_NPE.
## What changes were proposed in this pull request?
Support` ALTER TABLE ADD COLUMNS (...) `syntax for Hive serde and some datasource tables.
In this PR, we consider a few aspects:
1. View is not supported for `ALTER ADD COLUMNS`
2. Since tables created in SparkSQL with Hive DDL syntax will populate table properties with schema information, we need make sure the consistency of the schema before and after ALTER operation in order for future use.
3. For embedded-schema type of format, such as `parquet`, we need to make sure that the predicate on the newly-added columns can be evaluated properly, or pushed down properly. In case of the data file does not have the columns for the newly-added columns, such predicates should return as if the column values are NULLs.
4. For datasource table, this feature does not support the following:
4.1 TEXT format, since there is only one default column `value` is inferred for text format data.
4.2 ORC format, since SparkSQL native ORC reader does not support the difference between user-specified-schema and inferred schema from ORC files.
4.3 Third party datasource types that implements RelationProvider, including the built-in JDBC format, since different implementations by the vendors may have different ways to dealing with schema.
4.4 Other datasource types, such as `parquet`, `json`, `csv`, `hive` are supported.
5. Column names being added can not be duplicate of any existing data column or partition column names. Case sensitivity is taken into consideration according to the sql configuration.
6. This feature also supports In-Memory catalog, while Hive support is turned off.
## How was this patch tested?
Add new test cases
Author: Xin Wu <xinwu@us.ibm.com>
Closes#16626 from xwu0226/alter_add_columns.
## What changes were proposed in this pull request?
1. Improve documentation for class `Cost` and `JoinReorderDP` and method `buildJoin()`.
2. Change code structure of `buildJoin()` to make the logic clearer.
3. Add a debug-level log to record information for join reordering, including time cost, the number of items and the number of plans in memo.
## How was this patch tested?
Not related.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17353 from wzhfy/reorderFollow.
### What changes were proposed in this pull request?
SessionCatalog API setCurrentDatabase does not set the current database of the underlying ExternalCatalog. Thus, weird errors could come in the test suites after we call reset. We need to fix it.
So far, have not found the direct impact in the other code paths because we expect all the SessionCatalog APIs should always use the current database value we managed, unless some of code paths skip it. Thus, we fix it in the test-only function reset().
### How was this patch tested?
Multiple test case failures are observed in mvn and add a test case in SessionCatalogSuite.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17354 from gatorsmile/useDB.
## What changes were proposed in this pull request?
Currently JSON and CSV have exactly the same logic about handling bad records, this PR tries to abstract it and put it in a upper level to reduce code duplication.
The overall idea is, we make the JSON and CSV parser to throw a BadRecordException, then the upper level, FailureSafeParser, handles bad records according to the parse mode.
Behavior changes:
1. with PERMISSIVE mode, if the number of tokens doesn't match the schema, previously CSV parser will treat it as a legal record and parse as many tokens as possible. After this PR, we treat it as an illegal record, and put the raw record string in a special column, but we still parse as many tokens as possible.
2. all logging is removed as they are not very useful in practice.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Wenchen Fan <cloud0fan@gmail.com>
Closes#17315 from cloud-fan/bad-record2.
## What changes were proposed in this pull request?
A Bean serializer in `ExpressionEncoder` could change values when Beans having NULL. A concrete example is as follows;
```
scala> :paste
class Outer extends Serializable {
private var cls: Inner = _
def setCls(c: Inner): Unit = cls = c
def getCls(): Inner = cls
}
class Inner extends Serializable {
private var str: String = _
def setStr(s: String): Unit = str = str
def getStr(): String = str
}
scala> Seq("""{"cls":null}""", """{"cls": {"str":null}}""").toDF().write.text("data")
scala> val encoder = Encoders.bean(classOf[Outer])
scala> val schema = encoder.schema
scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder)
scala> df.show
+------+
| cls|
+------+
|[null]|
| null|
+------+
scala> df.map(x => x)(encoder).show()
+------+
| cls|
+------+
|[null]|
|[null]| // <-- Value changed
+------+
```
This is because the Bean serializer does not have the NULL-check expressions that the serializer of Scala's product types has. Actually, this value change does not happen in Scala's product types;
```
scala> :paste
case class Outer(cls: Inner)
case class Inner(str: String)
scala> val encoder = Encoders.product[Outer]
scala> val schema = encoder.schema
scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder)
scala> df.show
+------+
| cls|
+------+
|[null]|
| null|
+------+
scala> df.map(x => x)(encoder).show()
+------+
| cls|
+------+
|[null]|
| null|
+------+
```
This pr added the NULL-check expressions in Bean serializer along with the serializer of Scala's product types.
## How was this patch tested?
Added tests in `JavaDatasetSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17347 from maropu/SPARK-19980.
## What changes were proposed in this pull request?
After sort merge join for inner join, now we only keep left key ordering. However, after inner join, right key has the same value and order as left key. So if we need another smj on right key, we will unnecessarily add a sort which causes additional cost.
As a more complicated example, A join B on A.key = B.key join C on B.key = C.key join D on A.key = D.key. We will unnecessarily add a sort on B.key when join {A, B} and C, and add a sort on A.key when join {A, B, C} and D.
To fix this, we need to propagate all sorted information (equivalent expressions) from bottom up through `outputOrdering` and `SortOrder`.
## How was this patch tested?
Test cases are added.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17339 from wzhfy/sortEnhance.
## What changes were proposed in this pull request?
update `StatFunctions.multipleApproxQuantiles` to handle NaN/null
## How was this patch tested?
existing tests and added tests
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Closes#16971 from zhengruifeng/quantiles_nan.
## What changes were proposed in this pull request?
Star schema consists of one or more fact tables referencing a number of dimension tables. In general, queries against star schema are expected to run fast because of the established RI constraints among the tables. This design proposes a join reordering based on natural, generally accepted heuristics for star schema queries:
- Finds the star join with the largest fact table and places it on the driving arm of the left-deep join. This plan avoids large tables on the inner, and thus favors hash joins.
- Applies the most selective dimensions early in the plan to reduce the amount of data flow.
The design document was included in SPARK-17791.
Link to the google doc: [StarSchemaDetection](https://docs.google.com/document/d/1UAfwbm_A6wo7goHlVZfYK99pqDMEZUumi7pubJXETEA/edit?usp=sharing)
## How was this patch tested?
A new test suite StarJoinSuite.scala was implemented.
Author: Ioana Delaney <ioanamdelaney@gmail.com>
Closes#15363 from ioana-delaney/starJoinReord2.
## What changes were proposed in this pull request?
This PR proposes to support an array of struct type in `to_json` as below:
```scala
import org.apache.spark.sql.functions._
val df = Seq(Tuple1(Tuple1(1) :: Nil)).toDF("a")
df.select(to_json($"a").as("json")).show()
```
```
+----------+
| json|
+----------+
|[{"_1":1}]|
+----------+
```
Currently, it throws an exception as below (a newline manually inserted for readability):
```
org.apache.spark.sql.AnalysisException: cannot resolve 'structtojson(`array`)' due to data type
mismatch: structtojson requires that the expression is a struct expression.;;
```
This allows the roundtrip with `from_json` as below:
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val df = Seq("""[{"a":1}, {"a":2}]""").toDF("json").select(from_json($"json", schema).as("array"))
df.show()
// Read back.
df.select(to_json($"array").as("json")).show()
```
```
+----------+
| array|
+----------+
|[[1], [2]]|
+----------+
+-----------------+
| json|
+-----------------+
|[{"a":1},{"a":2}]|
+-----------------+
```
Also, this PR proposes to rename from `StructToJson` to `StructsToJson ` and `JsonToStruct` to `JsonToStructs`.
## How was this patch tested?
Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite` for Scala, doctest for Python and test in `test_sparkSQL.R` for R.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17192 from HyukjinKwon/SPARK-19849.
## What changes were proposed in this pull request?
When a key does not get any new data in `mapGroupsWithState`, the mapping function is never called on it. So we need a timeout feature that calls the function again in such cases, so that the user can decide whether to continue waiting or clean up (remove state, save stuff externally, etc.).
Timeouts can be either based on processing time or event time. This JIRA is for processing time, but defines the high level API design for both. The usage would look like this.
```
def stateFunction(key: K, value: Iterator[V], state: KeyedState[S]): U = {
...
state.setTimeoutDuration(10000)
...
}
dataset // type is Dataset[T]
.groupByKey[K](keyingFunc) // generates KeyValueGroupedDataset[K, T]
.mapGroupsWithState[S, U](
func = stateFunction,
timeout = KeyedStateTimeout.withProcessingTime) // returns Dataset[U]
```
Note the following design aspects.
- The timeout type is provided as a param in mapGroupsWithState as a parameter global to all the keys. This is so that the planner knows this at planning time, and accordingly optimize the execution based on whether to saves extra info in state or not (e.g. timeout durations or timestamps).
- The exact timeout duration is provided inside the function call so that it can be customized on a per key basis.
- When the timeout occurs for a key, the function is called with no values, and KeyedState.isTimingOut() set to true.
- The timeout is reset for key every time the function is called on the key, that is, when the key has new data, or the key has timed out. So the user has to set the timeout duration everytime the function is called, otherwise there will not be any timeout set.
Guarantees provided on timeout of key, when timeout duration is D ms:
- Timeout will never be called before real clock time has advanced by D ms
- Timeout will be called eventually when there is a trigger with any data in it (i.e. after D ms). So there is a no strict upper bound on when the timeout would occur. For example, if there is no data in the stream (for any key) for a while, then the timeout will not be hit.
Implementation details:
- Added new param to `mapGroupsWithState` for timeout
- Added new method to `StateStore` to filter data based on timeout timestamp
- Changed the internal map type of `HDFSBackedStateStore` from Java's `HashMap` to `ConcurrentHashMap` as the latter allows weakly-consistent fail-safe iterators on the map data. See comments in code for more details.
- Refactored logic of `MapGroupsWithStateExec` to
- Save timeout info to state store for each key that has data.
- Then, filter states that should be timed out based on the current batch processing timestamp.
- Moved KeyedState for `o.a.s.sql` to `o.a.s.sql.streaming`. I remember that this was a feedback in the MapGroupsWithState PR that I had forgotten to address.
## How was this patch tested?
New unit tests in
- MapGroupsWithStateSuite for timeouts.
- StateStoreSuite for new APIs in StateStore.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#17179 from tdas/mapgroupwithstate-timeout.
## What changes were proposed in this pull request?
If case classes have circular references below, it throws StackOverflowError;
```
scala> :pasge
case class classA(i: Int, cls: classB)
case class classB(cls: classA)
scala> Seq(classA(0, null)).toDS()
java.lang.StackOverflowError
at scala.reflect.internal.Symbols$Symbol.info(Symbols.scala:1494)
at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.scala$reflect$runtime$SynchronizedSymbols$SynchronizedSymbol$$super$info(JavaMirrors.scala:66)
at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.gilSynchronizedIfNotThreadsafe(SynchronizedSymbols.scala:123)
at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.gilSynchronizedIfNotThreadsafe(JavaMirrors.scala:66)
at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.info(SynchronizedSymbols.scala:127)
at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.info(JavaMirrors.scala:66)
at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:48)
at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
```
This pr added code to throw UnsupportedOperationException in that case as follows;
```
scala> :paste
case class A(cls: B)
case class B(cls: A)
scala> Seq(A(null)).toDS()
java.lang.UnsupportedOperationException: cannot have circular references in class, but got the circular reference of class B
at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:627)
at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:644)
at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:632)
at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
at scala.collection.immutable.List.foreach(List.scala:381)
at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
```
## How was this patch tested?
Added tests in `DatasetSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17318 from maropu/SPARK-19896.
## What changes were proposed in this pull request?
We have some concerns about removing size in the cost model [in the previous pr](https://github.com/apache/spark/pull/17240). It's a tradeoff between code structure and algorithm completeness. I tend to keep the size and thus create this new pr without changing cost model.
What this pr does:
1. We only consider consecutive inner joinable items, thus excluding cartesian products in reordering procedure. This significantly reduces the search space and memory overhead of memo. Otherwise every combination of items will exist in the memo.
2. This pr also includes a bug fix: if a leaf item is a project(_, child), current solution will miss the project.
## How was this patch tested?
Added test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#17286 from wzhfy/joinReorder3.
## What changes were proposed in this pull request?
This pr added entries in `FunctionRegistry` and supported `from_json` in SQL.
## How was this patch tested?
Added tests in `JsonFunctionsSuite` and `SQLQueryTestSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17320 from maropu/SPARK-19967.
## What changes were proposed in this pull request?
Allows null values of the pivot column to be included in the pivot values list without throwing NPE
Note this PR was made as an alternative to #17224 but preserves the two phase aggregate operation that is needed for good performance.
## How was this patch tested?
Additional unit test
Author: Andrew Ray <ray.andrew@gmail.com>
Closes#17226 from aray/pivot-null.
## What changes were proposed in this pull request?
Currently `SessionCatalogSuite` is only for `InMemoryCatalog`, there is no suite for `HiveExternalCatalog`.
And there are some ddl function is not proper to test in `ExternalCatalogSuite`, because some logic are not full implement in `ExternalCatalog`, these ddl functions are full implement in `SessionCatalog`(e.g. merge the same logic from `ExternalCatalog` up to `SessionCatalog` ).
It is better to test it in `SessionCatalogSuite` for this situation.
So we should add a test suite for `SessionCatalog` with `HiveExternalCatalog`
The main change is that in `SessionCatalogSuite` add two functions:
`withBasicCatalog` and `withEmptyCatalog`
And replace the code like `val catalog = new SessionCatalog(newBasicCatalog)` with above two functions
## How was this patch tested?
add `HiveExternalSessionCatalogSuite`
Author: windpiger <songjun@outlook.com>
Closes#17287 from windpiger/sessioncatalogsuit.
### What changes were proposed in this pull request?
Specifying the table schema in DDL formats is needed for different scenarios. For example,
- [specifying the schema in SQL function `from_json` using DDL formats](https://issues.apache.org/jira/browse/SPARK-19637), which is suggested by marmbrus ,
- [specifying the customized JDBC data types](https://github.com/apache/spark/pull/16209).
These two PRs need users to use the JSON format to specify the table schema. This is not user friendly.
This PR is to provide a `parseTableSchema` API in `ParserInterface`.
### How was this patch tested?
Added a test suite `TableSchemaParserSuite`
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17171 from gatorsmile/parseDDLStmt.
## What changes were proposed in this pull request?
The current master throws `StackOverflowError` in `createDataFrame`/`createDataset` if bean has one's own class in fields;
```
public class SelfClassInFieldBean implements Serializable {
private SelfClassInFieldBean child;
...
}
```
This pr added code to throw `UnsupportedOperationException` in that case as soon as possible.
## How was this patch tested?
Added tests in `JavaDataFrameSuite` and `JavaDatasetSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17188 from maropu/SPARK-19751.
## What changes were proposed in this pull request?
unify a exception erro msg for dropdatabase when the database still have some tables for HiveExternalCatalog and InMemoryCatalog
## How was this patch tested?
N/A
Author: windpiger <songjun@outlook.com>
Closes#17305 from windpiger/unifyErromsg.
## What issue does this PR address ?
Jira: https://issues.apache.org/jira/browse/SPARK-13450
In `SortMergeJoinExec`, rows of the right relation having the same value for a join key are buffered in-memory. In case of skew, this causes OOMs (see comments in SPARK-13450 for more details). Heap dump from a failed job confirms this : https://issues.apache.org/jira/secure/attachment/12846382/heap-dump-analysis.png . While its possible to increase the heap size to workaround, Spark should be resilient to such issues as skews can happen arbitrarily.
## Change proposed in this pull request
- Introduces `ExternalAppendOnlyUnsafeRowArray`
- It holds `UnsafeRow`s in-memory upto a certain threshold.
- After the threshold is hit, it switches to `UnsafeExternalSorter` which enables spilling of the rows to disk. It does NOT sort the data.
- Allows iterating the array multiple times. However, any alteration to the array (using `add` or `clear`) will invalidate the existing iterator(s)
- `WindowExec` was already using `UnsafeExternalSorter` to support spilling. Changed it to use the new array
- Changed `SortMergeJoinExec` to use the new array implementation
- NOTE: I have not changed FULL OUTER JOIN to use this new array implementation. Changing that will need more surgery and I will rather put up a separate PR for that once this gets in.
- Changed `CartesianProductExec` to use the new array implementation
#### Note for reviewers
The diff can be divided into 3 parts. My motive behind having all the changes in a single PR was to demonstrate that the API is sane and supports 2 use cases. If reviewing as 3 separate PRs would help, I am happy to make the split.
## How was this patch tested ?
#### Unit testing
- Added unit tests `ExternalAppendOnlyUnsafeRowArray` to validate all its APIs and access patterns
- Added unit test for `SortMergeExec`
- with and without spill for inner join, left outer join, right outer join to confirm that the spill threshold config behaves as expected and output is as expected.
- This PR touches the scanning logic in `SortMergeExec` for _all_ joins (except FULL OUTER JOIN). However, I expect existing test cases to cover that there is no regression in correctness.
- Added unit test for `WindowExec` to check behavior of spilling and correctness of results.
#### Stress testing
- Confirmed that OOM is gone by running against a production job which used to OOM
- Since I cannot share details about prod workload externally, created synthetic data to mimic the issue. Ran before and after the fix to demonstrate the issue and query success with this PR
Generating the synthetic data
```
./bin/spark-shell --driver-memory=6G
import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()
hc.sql("DROP TABLE IF EXISTS spark_13450_large_table").collect
hc.sql("DROP TABLE IF EXISTS spark_13450_one_row_table").collect
val df1 = (0 until 1).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df1.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_one_row_table")
val df2 = (0 until 3000000).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df2.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_large_table")
```
Ran this against trunk VS local build with this PR. OOM repros with trunk and with the fix this query runs fine.
```
./bin/spark-shell --driver-java-options="-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/spark.driver.heapdump.hprof"
import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()
hc.sql("SET spark.sql.autoBroadcastJoinThreshold=1")
hc.sql("SET spark.sql.sortMergeJoinExec.buffer.spill.threshold=10000")
hc.sql("DROP TABLE IF EXISTS spark_13450_result").collect
hc.sql("""
CREATE TABLE spark_13450_result
AS
SELECT
a.i AS a_i, a.j AS a_j, a.str1 AS a_str1, a.str2 AS a_str2,
b.i AS b_i, b.j AS b_j, b.str1 AS b_str1, b.str2 AS b_str2
FROM
spark_13450_one_row_table a
JOIN
spark_13450_large_table b
ON
a.i=b.i AND
a.j=b.j
""")
```
## Performance comparison
### Macro-benchmark
I ran a SMB join query over two real world tables (2 trillion rows (40 TB) and 6 million rows (120 GB)). Note that this dataset does not have skew so no spill happened. I saw improvement in CPU time by 2-4% over version without this PR. This did not add up as I was expected some regression. I think allocating array of capacity of 128 at the start (instead of starting with default size 16) is the sole reason for the perf. gain : https://github.com/tejasapatil/spark/blob/SPARK-13450_smb_buffer_oom/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala#L43 . I could remove that and rerun, but effectively the change will be deployed in this form and I wanted to see the effect of it over large workload.
### Micro-benchmark
Two types of benchmarking can be found in `ExternalAppendOnlyUnsafeRowArrayBenchmark`:
[A] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `ArrayBuffer` when all rows fit in-memory and there is no spill
```
Array with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
ArrayBuffer 7821 / 7941 33.5 29.8 1.0X
ExternalAppendOnlyUnsafeRowArray 8798 / 8819 29.8 33.6 0.9X
Array with 30000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
ArrayBuffer 19200 / 19206 25.6 39.1 1.0X
ExternalAppendOnlyUnsafeRowArray 19558 / 19562 25.1 39.8 1.0X
Array with 100000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
ArrayBuffer 5949 / 6028 17.2 58.1 1.0X
ExternalAppendOnlyUnsafeRowArray 6078 / 6138 16.8 59.4 1.0X
```
[B] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `UnsafeExternalSorter` when there is spilling of data
```
Spilling with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter 9239 / 9470 28.4 35.2 1.0X
ExternalAppendOnlyUnsafeRowArray 8857 / 8909 29.6 33.8 1.0X
Spilling with 10000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter 4 / 5 39.3 25.5 1.0X
ExternalAppendOnlyUnsafeRowArray 5 / 6 29.8 33.5 0.8X
```
Author: Tejas Patil <tejasp@fb.com>
Closes#16909 from tejasapatil/SPARK-13450_smb_buffer_oom.
## What changes were proposed in this pull request?
We should restrict the nested level of a view, to avoid stack overflow exception during the view resolution.
## How was this patch tested?
Add new test case in `SQLViewSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#17241 from jiangxb1987/view-depth.
## What changes were proposed in this pull request?
When dynamic partition value is null or empty string, we should write the data to a directory like `a=__HIVE_DEFAULT_PARTITION__`, when we read the data back, we should respect this special directory name and treat it as null.
This is the same behavior of impala, see https://issues.apache.org/jira/browse/IMPALA-252
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17277 from cloud-fan/partition.
## What changes were proposed in this pull request?
As timezone setting can also affect partition values, it works for all formats, we should make it clear.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#17281 from ueshin/issues/SPARK-19817.
## What changes were proposed in this pull request?
This PR fixes the following problem:
````
Seq((1, 2)).toDF("a1", "a2").createOrReplaceTempView("a")
Seq[(java.lang.Integer, java.lang.Integer)]((1, null)).toDF("b1", "b2").createOrReplaceTempView("b")
// The expected result is 1 row of (1,2) as shown in the next statement.
sql("select * from a where a1 not in (select b1 from b where b2 = a2)").show
+---+---+
| a1| a2|
+---+---+
+---+---+
sql("select * from a where a1 not in (select b1 from b where b2 = 2)").show
+---+---+
| a1| a2|
+---+---+
| 1| 2|
+---+---+
````
There are a number of scenarios to consider:
1. When the correlated predicate yields a match (i.e., B.B2 = A.A2)
1.1. When the NOT IN expression yields a match (i.e., A.A1 = B.B1)
1.2. When the NOT IN expression yields no match (i.e., A.A1 = B.B1 returns false)
1.3. When A.A1 is null
1.4. When B.B1 is null
1.4.1. When A.A1 is not null
1.4.2. When A.A1 is null
2. When the correlated predicate yields no match (i.e.,B.B2 = A.A2 is false or unknown)
2.1. When B.B2 is null and A.A2 is null
2.2. When B.B2 is null and A.A2 is not null
2.3. When the value of A.A2 does not match any of B.B2
````
A.A1 A.A2 B.B1 B.B2
----- ----- ----- -----
1 1 1 1 (1.1)
2 1 (1.2)
null 1 (1.3)
1 3 null 3 (1.4.1)
null 3 (1.4.2)
1 null 1 null (2.1)
null 2 (2.2 & 2.3)
````
We can divide the evaluation of the above correlated NOT IN subquery into 2 groups:-
Group 1: The rows in A when there is a match from the correlated predicate (A.A1 = B.B1)
In this case, the result of the subquery is not empty and the semantics of the NOT IN depends solely on the evaluation of the equality comparison of the columns of NOT IN, i.e., A1 = B1, which says
- If A.A1 is null, the row is filtered (1.3 and 1.4.2)
- If A.A1 = B.B1, the row is filtered (1.1)
- If B.B1 is null, any rows of A in the same group (A.A2 = B.B2) is filtered (1.4.1 & 1.4.2)
- Otherwise, the row is qualified.
Hence, in this group, the result is the row from (1.2).
Group 2: The rows in A when there is no match from the correlated predicate (A.A2 = B.B2)
In this case, all the rows in A, including the rows where A.A1, are qualified because the subquery returns an empty set and by the semantics of the NOT IN, all rows from the parent side qualifies as the result set, that is, the rows from (2.1, 2.2 and 2.3).
In conclusion, the correct result set of the above query is
````
A.A1 A.A2
----- -----
2 1 (1.2)
1 null (2.1)
null 2 (2.2 & 2.3)
````
## How was this patch tested?
unit tests, regression tests, and new test cases focusing on the problem being fixed.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#17294 from nsyca/18966.
## What changes were proposed in this pull request?
The `RemoveRedundantAlias` rule can change the output attributes (the expression id's to be precise) of a query by eliminating the redundant alias producing them. This is no problem for a regular query, but can cause problems for correlated subqueries: The attributes produced by the subquery are used in the parent plan; changing them will break the parent plan.
This PR fixes this by wrapping a subquery in a `Subquery` top level node when it gets optimized. The `RemoveRedundantAlias` rule now recognizes `Subquery` and makes sure that the output attributes of the `Subquery` node are retained.
## How was this patch tested?
Added a test case to `RemoveRedundantAliasAndProjectSuite` and added a regression test to `SubquerySuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17278 from hvanhovell/SPARK-19933.
## What changes were proposed in this pull request?
Commit 4ce970d714 in accidentally broke the 2.10 build for Spark. This PR fixes this by simplifying the offending pattern match.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17288 from hvanhovell/SPARK-18874.
## What changes were proposed in this pull request?
We currently cannot use aliases in SQL function calls. This is inconvenient when you try to create a struct. This SQL query for example `select struct(1, 2) st`, will create a struct with column names `col1` and `col2`. This is even more problematic when we want to append a field to an existing struct. For example if we want to a field to struct `st` we would issue the following SQL query `select struct(st.*, 1) as st from src`, the result will be struct `st` with an a column with a non descriptive name `col3` (if `st` itself has 2 fields).
This PR proposes to change this by allowing the use of aliased expression in function parameters. For example `select struct(1 as a, 2 as b) st`, will create a struct with columns `a` & `b`.
## How was this patch tested?
Added a test to `ExpressionParserSuite` and added a test file for `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17245 from hvanhovell/SPARK-19850.
## What changes were proposed in this pull request?
This patch moves SQLConf from sql/core to sql/catalyst. To minimize the changes, the patch used type alias to still keep CatalystConf (as a type alias) and SimpleCatalystConf (as a concrete class that extends SQLConf).
Motivation for the change is that it is pretty weird to have SQLConf only in sql/core and then we have to duplicate config options that impact optimizer/analyzer in sql/catalyst using CatalystConf.
## How was this patch tested?
N/A
Author: Reynold Xin <rxin@databricks.com>
Closes#17285 from rxin/SPARK-19944.
## What changes were proposed in this pull request?
- Timestamp hashing is done as per [TimestampWritable.hashCode()](ff67cdda1c/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java (L406)) in Hive
- Interval hashing is done as per [HiveIntervalDayTime.hashCode()](ff67cdda1c/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java (L178)). Note that there are inherent differences in how Hive and Spark store intervals under the hood which limits the ability to be in completely sync with hive's hashing function. I have explained this in the method doc.
- Date type was already supported. This PR adds test for that.
## How was this patch tested?
Added unit tests
Author: Tejas Patil <tejasp@fb.com>
Closes#17062 from tejasapatil/SPARK-17495_time_related_types.
## What changes were proposed in this pull request?
In spark SQL, map type can't be used in equality test/comparison, and `Intersect`/`Except`/`Distinct` do need equality test for all columns, we should not allow map type in `Intersect`/`Except`/`Distinct`.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17236 from cloud-fan/map.
## Summary of changes
Add a new configuration option that allows Spark SQL to infer a case-sensitive schema from a Hive Metastore table's data files when a case-sensitive schema can't be read from the table properties.
- Add spark.sql.hive.caseSensitiveInferenceMode param to SQLConf
- Add schemaPreservesCase field to CatalogTable (set to false when schema can't
successfully be read from Hive table props)
- Perform schema inference in HiveMetastoreCatalog if schemaPreservesCase is
false, depending on spark.sql.hive.caseSensitiveInferenceMode
- Add alterTableSchema() method to the ExternalCatalog interface
- Add HiveSchemaInferenceSuite tests
- Refactor and move ParquetFileForamt.meregeMetastoreParquetSchema() as
HiveMetastoreCatalog.mergeWithMetastoreSchema
- Move schema merging tests from ParquetSchemaSuite to HiveSchemaInferenceSuite
[JIRA for this change](https://issues.apache.org/jira/browse/SPARK-19611)
## How was this patch tested?
The tests in ```HiveSchemaInferenceSuite``` should verify that schema inference is working as expected. ```ExternalCatalogSuite``` has also been extended to cover the new ```alterTableSchema()``` API.
Author: Budde <budde@amazon.com>
Closes#16944 from budde/SPARK-19611.
## What changes were proposed in this pull request?
If we create a external datasource table with a non-qualified location , we should qualified it to store in catalog.
```
CREATE TABLE t(a string)
USING parquet
LOCATION '/path/xx'
CREATE TABLE t1(a string, b string)
USING parquet
PARTITIONED BY(b)
LOCATION '/path/xx'
```
when we get the table from catalog, the location should be qualified, e.g.'file:/path/xxx'
## How was this patch tested?
unit test added
Author: windpiger <songjun@outlook.com>
Closes#17095 from windpiger/tablepathQualified.
## What changes were proposed in this pull request?
A follow up to SPARK-19859:
- extract the calculation of `delayMs` and reuse it.
- update EventTimeWatermarkExec
- use the correct `delayMs` in EventTimeWatermark
## How was this patch tested?
Jenkins.
Author: uncleGen <hustyugm@gmail.com>
Closes#17221 from uncleGen/SPARK-19859.
Forking a newSession() from SparkSession currently makes a new SparkSession that does not retain SessionState (i.e. temporary tables, SQL config, registered functions etc.) This change adds a method cloneSession() which creates a new SparkSession with a copy of the parent's SessionState.
Subsequent changes to base session are not propagated to cloned session, clone is independent after creation.
If the base is changed after clone has been created, say user registers new UDF, then the new UDF will not be available inside the clone. Same goes for configs and temp tables.
Unit tests
Author: Kunal Khamar <kkhamar@outlook.com>
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16826 from kunalkhamar/fork-sparksession.
## What changes were proposed in this pull request?
Add a output mode parameter to `flatMapGroupsWithState` and just define `mapGroupsWithState` as `flatMapGroupsWithState(Update)`.
`UnsupportedOperationChecker` is modified to disallow unsupported cases.
- Batch mapGroupsWithState or flatMapGroupsWithState is always allowed.
- For streaming (map/flatMap)GroupsWithState, see the following table:
| Operators | Supported Query Output Mode |
| ------------- | ------------- |
| flatMapGroupsWithState(Update) without aggregation | Update |
| flatMapGroupsWithState(Update) with aggregation | None |
| flatMapGroupsWithState(Append) without aggregation | Append |
| flatMapGroupsWithState(Append) before aggregation | Append, Update, Complete |
| flatMapGroupsWithState(Append) after aggregation | None |
| Multiple flatMapGroupsWithState(Append)s | Append |
| Multiple mapGroupsWithStates | None |
| Mxing mapGroupsWithStates and flatMapGroupsWithStates | None |
| Other cases of multiple flatMapGroupsWithState | None |
## How was this patch tested?
The added unit tests. Here are the tests related to (map/flatMap)GroupsWithState:
```
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation: supported (1 millisecond)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Append)s on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation: supported (0 milliseconds)
[info] - batch plan - flatMapGroupsWithState - multiple flatMapGroupsWithState(Update)s on batch relation: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in update mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in append mode: not supported (7 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation without aggregation in complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Append mode: not supported (11 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Update mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation with aggregation in Complete mode: not supported (5 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation without aggregation in update mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Append mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Update mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation before aggregation in Complete mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on streaming relation after aggregation in Update mode: not supported (4 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on streaming relation in complete mode: not supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Append output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Append) on batch relation inside streaming relation in Update output mode: supported (1 millisecond)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Append output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - flatMapGroupsWithState(Update) on batch relation inside streaming relation in Update output mode: supported (0 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - multiple flatMapGroupsWithStates on streaming relation and all are in append mode: supported (2 milliseconds)
[info] - streaming plan - flatMapGroupsWithState - multiple flatMapGroupsWithStates on s streaming relation but some are not in append mode: not supported (7 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in append mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation without aggregation in complete mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Append mode: not supported (6 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Update mode: not supported (3 milliseconds)
[info] - streaming plan - mapGroupsWithState - mapGroupsWithState on streaming relation with aggregation in Complete mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - multiple mapGroupsWithStates on streaming relation and all are in append mode: not supported (4 milliseconds)
[info] - streaming plan - mapGroupsWithState - mixing mapGroupsWithStates and flatMapGroupsWithStates on streaming relation: not supported (4 milliseconds)
```
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17197 from zsxwing/mapgroups-check.
## What changes were proposed in this pull request?
Fix for SQL round function that modifies original column when underlying data frame is created from a local product.
import org.apache.spark.sql.functions._
case class NumericRow(value: BigDecimal)
val df = spark.createDataFrame(Seq(NumericRow(BigDecimal("1.23456789"))))
df.show()
+--------------------+
| value|
+--------------------+
|1.234567890000000000|
+--------------------+
df.withColumn("value_rounded", round('value)).show()
// before
+--------------------+-------------+
| value|value_rounded|
+--------------------+-------------+
|1.000000000000000000| 1|
+--------------------+-------------+
// after
+--------------------+-------------+
| value|value_rounded|
+--------------------+-------------+
|1.234567890000000000| 1|
+--------------------+-------------+
## How was this patch tested?
New unit test added to existing suite `org.apache.spark.sql.MathFunctionsSuite`
Author: Wojtek Szymanski <wk.szymanski@gmail.com>
Closes#17075 from wojtek-szymanski/SPARK-19727.
### What changes were proposed in this pull request?
Observed by felixcheung in https://github.com/apache/spark/pull/16739, when users use the shuffle-enabled `repartition` API, they expect the partition they got should be the exact number they provided, even if they call shuffle-disabled `coalesce` later.
Currently, `CollapseRepartition` rule does not consider whether shuffle is enabled or not. Thus, we got the following unexpected result.
```Scala
val df = spark.range(0, 10000, 1, 5)
val df2 = df.repartition(10)
assert(df2.coalesce(13).rdd.getNumPartitions == 5)
assert(df2.coalesce(7).rdd.getNumPartitions == 5)
assert(df2.coalesce(3).rdd.getNumPartitions == 3)
```
This PR is to fix the issue. We preserve shuffle-enabled Repartition.
### How was this patch tested?
Added a test case
Author: Xiao Li <gatorsmile@gmail.com>
Closes#16933 from gatorsmile/CollapseRepartition.
## What changes were proposed in this pull request?
Since we have a `View` node now, we can remove the view identifier in `SubqueryAlias`, which was used to indicate a view node before.
## How was this patch tested?
Update the related test cases.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#17210 from jiangxb1987/SubqueryAlias.
## What changes were proposed in this pull request?
Reorder the joins using a dynamic programming algorithm (Selinger paper):
First we put all items (basic joined nodes) into level 1, then we build all two-way joins at level 2 from plans at level 1 (single items), then build all 3-way joins from plans at previous levels (two-way joins and single items), then 4-way joins ... etc, until we build all n-way joins and pick the best plan among them.
When building m-way joins, we only keep the best plan (with the lowest cost) for the same set of m items. E.g., for 3-way joins, we keep only the best plan for items {A, B, C} among plans (A J B) J C, (A J C) J B and (B J C) J A. Thus, the plans maintained for each level when reordering four items A, B, C, D are as follows:
```
level 1: p({A}), p({B}), p({C}), p({D})
level 2: p({A, B}), p({A, C}), p({A, D}), p({B, C}), p({B, D}), p({C, D})
level 3: p({A, B, C}), p({A, B, D}), p({A, C, D}), p({B, C, D})
level 4: p({A, B, C, D})
```
where p({A, B, C, D}) is the final output plan.
For cost evaluation, since physical costs for operators are not available currently, we use cardinalities and sizes to compute costs.
## How was this patch tested?
add test cases
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#17138 from wzhfy/joinReorder.
Previously, we were using the mirror of passed in `TypeTag` when reflecting to build an encoder. This fails when the outer class is built in (i.e. `Seq`'s default mirror is based on root classloader) but inner classes (i.e. `A` in `Seq[A]`) are defined in the REPL or a library.
This patch changes us to always reflect based on a mirror created using the context classloader.
Author: Michael Armbrust <michael@databricks.com>
Closes#17201 from marmbrus/replSeqEncoder.
## What changes were proposed in this pull request?
The new watermark should override the old one. Otherwise, we just pick up the first column which has a watermark, it may be unexpected.
## How was this patch tested?
The new test.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17199 from zsxwing/SPARK-19859.
## What changes were proposed in this pull request?
Jira : https://issues.apache.org/jira/browse/SPARK-19843
Created wrapper classes (`IntWrapper`, `LongWrapper`) to wrap the result of parsing (which are primitive types). In case of problem in parsing, the method would return a boolean.
## How was this patch tested?
- Added new unit tests
- Ran a prod job which had conversion from string -> int and verified the outputs
## Performance
Tiny regression when all strings are valid integers
```
conversion to int: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------
trunk 502 / 522 33.4 29.9 1.0X
SPARK-19843 493 / 503 34.0 29.4 1.0X
```
Huge gain when all strings are invalid integers
```
conversion to int: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------
trunk 33913 / 34219 0.5 2021.4 1.0X
SPARK-19843 154 / 162 108.8 9.2 220.0X
```
Author: Tejas Patil <tejasp@fb.com>
Closes#17184 from tejasapatil/SPARK-19843_is_numeric_maybe.
## What changes were proposed in this pull request?
This pr added entries in `FunctionRegistry` and supported `to_json` in SQL.
## How was this patch tested?
Added tests in `JsonFunctionsSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#16981 from maropu/SPARK-19637.
## What changes were proposed in this pull request?
1. support boolean type in binary expression estimation.
2. deal with compound Not conditions.
3. avoid convert BigInt/BigDecimal directly to double unless it's within range (0, 1).
4. reorganize test code.
## How was this patch tested?
modify related test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#17148 from wzhfy/fixFilter.
## What changes were proposed in this pull request?
Before this pr, LocalLimit/GlobalLimit/Sample propagates the same row count and column stats from its child, which is incorrect.
We can get the correct rowCount in Statistics for GlobalLimit/Sample whether cbo is enabled or not.
We don't know the rowCount for LocalLimit because we don't know the partition number at that time. Column stats should not be propagated because we don't know the distribution of columns after Limit or Sample.
## How was this patch tested?
Added test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#16696 from wzhfy/limitEstimation.
## What changes were proposed in this pull request?
Currently we don't explicitly forbid the following behaviors:
1. The statement CREATE VIEW AS INSERT INTO throws the following exception:
```
scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.hadoop.hive.ql.metadata.HiveException: at least one column must be specified for the table;
scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: The number of columns produced by the SELECT clause (num: `0`) does not match the number of column names specified by CREATE VIEW (num: `2`).;
```
2. The statement INSERT INTO view VALUES throws the following exception from checkAnalysis:
```
scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: Inserting into an RDD-based table is not allowed.;;
'InsertIntoTable View (`default`.`testView`, [a#16,b#17]), false, false
+- LocalRelation [col1#14, col2#15]
```
After this PR, the behavior changes to:
```
scala> spark.sql("CREATE VIEW testView AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO;
scala> spark.sql("CREATE VIEW testView(a, b) AS INSERT INTO tab VALUES (1, \"a\")")
org.apache.spark.sql.catalyst.parser.ParseException: Operation not allowed: CREATE VIEW ... AS INSERT INTO;
scala> spark.sql("INSERT INTO testView VALUES (1, \"a\")")
org.apache.spark.sql.AnalysisException: `default`.`testView` is a view, inserting into a view is not allowed;
```
## How was this patch tested?
Add a new test case in `SparkSqlParserSuite`;
Update the corresponding test case in `SQLViewSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#17125 from jiangxb1987/insert-with-view.
## What changes were proposed in this pull request?
Currently we treat the location of table/partition/database as URI string.
It will be safer if we can make the type of location as java.net.URI.
In this PR, there are following classes changes:
**1. CatalogDatabase**
```
case class CatalogDatabase(
name: String,
description: String,
locationUri: String,
properties: Map[String, String])
--->
case class CatalogDatabase(
name: String,
description: String,
locationUri: URI,
properties: Map[String, String])
```
**2. CatalogStorageFormat**
```
case class CatalogStorageFormat(
locationUri: Option[String],
inputFormat: Option[String],
outputFormat: Option[String],
serde: Option[String],
compressed: Boolean,
properties: Map[String, String])
---->
case class CatalogStorageFormat(
locationUri: Option[URI],
inputFormat: Option[String],
outputFormat: Option[String],
serde: Option[String],
compressed: Boolean,
properties: Map[String, String])
```
Before and After this PR, it is transparent for user, there is no change that the user should concern. The `String` to `URI` just happened in SparkSQL internally.
Here list some operation related location:
**1. whitespace in the location**
e.g. `/a/b c/d`
For both table location and partition location,
After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b c/d'` ,
then `DESC EXTENDED t ` show the location is `/a/b c/d`,
and the real path in the FileSystem also show `/a/b c/d`
**2. colon(:) in the location**
e.g. `/a/b:c/d`
For both table location and partition location,
when `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b:c/d'` ,
**In linux file system**
`DESC EXTENDED t ` show the location is `/a/b:c/d`,
and the real path in the FileSystem also show `/a/b:c/d`
**in HDFS** throw exception:
`java.lang.IllegalArgumentException: Pathname /a/b:c/d from hdfs://iZbp1151s8hbnnwriekxdeZ:9000/a/b:c/d is not a valid DFS filename.`
**while** After `INSERT INTO TABLE t PARTITION(a="a:b") SELECT 1`
then `DESC EXTENDED t ` show the location is `/xxx/a=a%3Ab`,
and the real path in the FileSystem also show `/xxx/a=a%3Ab`
**3. percent sign(%) in the location**
e.g. `/a/b%c/d`
For both table location and partition location,
After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b%c/d'` ,
then `DESC EXTENDED t ` show the location is `/a/b%c/d`,
and the real path in the FileSystem also show `/a/b%c/d`
**4. encoded(%25) in the location**
e.g. `/a/b%25c/d`
For both table location and partition location,
After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION '/a/b%25c/d'` ,
then `DESC EXTENDED t ` show the location is `/a/b%25c/d`,
and the real path in the FileSystem also show `/a/b%25c/d`
**while** After `INSERT INTO TABLE t PARTITION(a="%25") SELECT 1`
then `DESC EXTENDED t ` show the location is `/xxx/a=%2525`,
and the real path in the FileSystem also show `/xxx/a=%2525`
**Additionally**, except the location, there are two other factors will affect the location of the table/partition. one is the table name which does not allowed to have special characters, and the other is `partition name` which have the same actions with `partition value`, and `partition name` with special character situation has add some testcase and resolve a bug in [PR](https://github.com/apache/spark/pull/17173)
### Summary:
After `CREATE TABLE t... (PARTITIONED BY ...) LOCATION path`,
the path which we get from `DESC TABLE` and `real path in FileSystem` are all the same with the `CREATE TABLE` command(different filesystem has different action that allow what kind of special character to create the path, e.g. HDFS does not allow colon, but linux filesystem allow it ).
`DataBase` also have the same logic with `CREATE TABLE`
while if the `partition value` has some special character like `%` `:` `#` etc, then we will get the path with encoded `partition value` like `/xxx/a=A%25B` from `DESC TABLE` and `real path in FileSystem`
In this PR, the core change code is using `new Path(str).toUri` and `new Path(uri).toString`
which transfrom `str to uri `or `uri to str`.
for example:
```
val str = '/a/b c/d'
val uri = new Path(str).toUri --> '/a/b%20c/d'
val strFromUri = new Path(uri).toString -> '/a/b c/d'
```
when we restore table/partition from metastore, or get the location from `CREATE TABLE` command, we can use it as above to change string to uri `new Path(str).toUri `
## How was this patch tested?
unit test added.
The `current master branch` also `passed all the test cases` added in this PR by a litter change.
https://github.com/apache/spark/pull/17149/files#diff-b7094baa12601424a5d19cb930e3402fR1764
here `toURI` -> `toString` when test in master branch.
This can show that this PR is transparent for user.
Author: windpiger <songjun@outlook.com>
Closes#17149 from windpiger/changeStringToURI.
## What changes were proposed in this pull request?
This PR adds a new `Once` analysis rule batch consists of a single analysis rule `LookupFunctions` that performs simple existence check over `UnresolvedFunctions` without actually resolving them.
The benefit of this rule is that it doesn't require function arguments to be resolved first and therefore doesn't rely on relation resolution, which may incur potentially expensive partition/schema discovery cost.
Please refer to [SPARK-19737][1] for more details about the motivation.
## How was this patch tested?
New test case added in `AnalysisErrorSuite`.
[1]: https://issues.apache.org/jira/browse/SPARK-19737
Author: Cheng Lian <lian@databricks.com>
Closes#17168 from liancheng/spark-19737-lookup-functions.
## What changes were proposed in this pull request?
Hive hash to support Decimal datatype. [Hive internally normalises decimals](4ba713ccd8/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java (L307)) and I have ported that logic as-is to HiveHash.
## How was this patch tested?
Added unit tests
Author: Tejas Patil <tejasp@fb.com>
Closes#17056 from tejasapatil/SPARK-17495_decimal.
## What changes were proposed in this pull request?
This PR proposes to both,
**Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.**
Currently, it only reads the single row when the input is a json array. So, the codes below:
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = StructType(StructField("a", IntegerType) :: Nil)
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show()
```
prints
```
+--------------------+
|jsontostruct(struct)|
+--------------------+
| [1]|
+--------------------+
```
This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements
```
+--------------------+
|jsontostruct(struct)|
+--------------------+
| null|
+--------------------+
```
**Support json arrays in `from_json` with `ArrayType` as the schema.**
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show()
```
prints
```
+-------------------+
|jsontostruct(array)|
+-------------------+
| [[1], [2]]|
+-------------------+
```
## How was this patch tested?
Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16929 from HyukjinKwon/disallow-array.
## What changes were proposed in this pull request?
This pr is to support Seq, Map, and Struct in functions.lit; it adds a new IF named `lit2` with `TypeTag` for avoiding type erasure.
## How was this patch tested?
Added tests in `LiteralExpressionSuite`
Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#16610 from maropu/SPARK-19254.
## What changes were proposed in this pull request?
This is a follow-up pr of #16308 and #16750.
This pr enables timezone support in partition values.
We should use `timeZone` option introduced at #16750 to parse/format partition values of the `TimestampType`.
For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT` which will be used for partition values, the values written by the default timezone option, which is `"GMT"` because the session local timezone is `"GMT"` here, are:
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")
scala> val df = Seq((1, new java.sql.Timestamp(1451606400000L))).toDF("i", "ts")
df: org.apache.spark.sql.DataFrame = [i: int, ts: timestamp]
scala> df.show()
+---+-------------------+
| i| ts|
+---+-------------------+
| 1|2016-01-01 00:00:00|
+---+-------------------+
scala> df.write.partitionBy("ts").save("/path/to/gmtpartition")
```
```sh
$ ls /path/to/gmtpartition/
_SUCCESS ts=2016-01-01 00%3A00%3A00
```
whereas setting the option to `"PST"`, they are:
```scala
scala> df.write.option("timeZone", "PST").partitionBy("ts").save("/path/to/pstpartition")
```
```sh
$ ls /path/to/pstpartition/
_SUCCESS ts=2015-12-31 16%3A00%3A00
```
We can properly read the partition values if the session local timezone and the timezone of the partition values are the same:
```scala
scala> spark.read.load("/path/to/gmtpartition").show()
+---+-------------------+
| i| ts|
+---+-------------------+
| 1|2016-01-01 00:00:00|
+---+-------------------+
```
And even if the timezones are different, we can properly read the values with setting corrent timezone option:
```scala
// wrong result
scala> spark.read.load("/path/to/pstpartition").show()
+---+-------------------+
| i| ts|
+---+-------------------+
| 1|2015-12-31 16:00:00|
+---+-------------------+
// correct result
scala> spark.read.option("timeZone", "PST").load("/path/to/pstpartition").show()
+---+-------------------+
| i| ts|
+---+-------------------+
| 1|2016-01-01 00:00:00|
+---+-------------------+
```
## How was this patch tested?
Existing tests and added some tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#17053 from ueshin/issues/SPARK-18939.
## What changes were proposed in this pull request?
When we resolve inline tables in analyzer, we will evaluate the expressions of inline tables.
When it evaluates a `TimeZoneAwareExpression` expression, an error will happen because the `TimeZoneAwareExpression` is not associated with timezone yet.
So we need to resolve these `TimeZoneAwareExpression`s with time zone when resolving inline tables.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#17114 from viirya/resolve-timeawareexpr-inline-table.
## What changes were proposed in this pull request?
`MetastoreRelation` is used to represent table relation for hive tables, and provides some hive related information. We will resolve `SimpleCatalogRelation` to `MetastoreRelation` for hive tables, which is unnecessary as these 2 are the same essentially. This PR merges `SimpleCatalogRelation` and `MetastoreRelation`
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17015 from cloud-fan/table-relation.
## What changes were proposed in this pull request?
This PR proposes to fix the lint-breaks as below:
```
[ERROR] src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.network.buffer.ManagedBuffer.
[ERROR] src/main/java/org/apache/spark/unsafe/types/UTF8String.java:[156,10] (modifier) ModifierOrder: 'Nonnull' annotation modifier does not precede non-annotation modifiers.
[ERROR] src/main/java/org/apache/spark/SparkFirehoseListener.java:[122] (sizes) LineLength: Line is longer than 100 characters (found 105).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java:[164,78] (coding) OneStatementPerLine: Only one statement per line allowed.
[ERROR] src/test/java/test/org/apache/spark/JavaAPISuite.java:[1157] (sizes) LineLength: Line is longer than 100 characters (found 121).
[ERROR] src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java:[149] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/test/java/test/org/apache/spark/streaming/Java8APISuite.java:[146] (sizes) LineLength: Line is longer than 100 characters (found 122).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[32,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.Time.
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[611] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[1317] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java:[91] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[113] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[164] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[212] (sizes) LineLength: Line is longer than 100 characters (found 114).
[ERROR] src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java:[36] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java:[26,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[20,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[94] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.api.java.UDF1.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[72] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java:[121] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaRDD.
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaSparkContext.
```
## How was this patch tested?
Manually via
```bash
./dev/lint-java
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17072 from HyukjinKwon/java-lint.
## What changes were proposed in this pull request?
This is a follow-up of https://github.com/apache/spark/pull/16395. It fixes some code style issues, naming issues, some missing cases in pattern match, etc.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#17065 from cloud-fan/follow-up.
### What changes were proposed in this pull request?
As explained in Hive JIRA https://issues.apache.org/jira/browse/HIVE-12224, HOLD_DDLTIME was broken as soon as it landed. Hive 2.0 removes HOLD_DDLTIME from the API. In Spark SQL, we always set it to FALSE. Like Hive, we should also remove it from our Catalog APIs.
### How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Closes#17063 from gatorsmile/removalHoldDDLTime.
## What changes were proposed in this pull request?
Currently we can only check the estimated stats in logical plans by debugging. We need to provide an easier and more efficient way for developers/users.
In this pr, we add EXPLAIN COST command to show stats in the optimized logical plan.
E.g.
```
spark-sql> EXPLAIN COST select count(1) from store_returns;
...
== Optimized Logical Plan ==
Aggregate [count(1) AS count(1)#24L], Statistics(sizeInBytes=16.0 B, rowCount=1, isBroadcastable=false)
+- Project, Statistics(sizeInBytes=4.3 GB, rowCount=5.76E+8, isBroadcastable=false)
+- Relation[sr_returned_date_sk#3,sr_return_time_sk#4,sr_item_sk#5,sr_customer_sk#6,sr_cdemo_sk#7,sr_hdemo_sk#8,sr_addr_sk#9,sr_store_sk#10,sr_reason_sk#11,sr_ticket_number#12,sr_return_quantity#13,sr_return_amt#14,sr_return_tax#15,sr_return_amt_inc_tax#16,sr_fee#17,sr_return_ship_cost#18,sr_refunded_cash#19,sr_reversed_charge#20,sr_store_credit#21,sr_net_loss#22] parquet, Statistics(sizeInBytes=28.6 GB, rowCount=5.76E+8, isBroadcastable=false)
...
```
## How was this patch tested?
Add test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#16594 from wzhfy/showStats.
## What changes were proposed in this pull request?
Fixed the line ending of `FilterEstimation.scala` (It's still using `\n\r`). Also improved the tests to cover the cases where the literals are on the left side of a binary operator.
## How was this patch tested?
Existing unit tests.
Author: Shuai Lin <linshuai2012@gmail.com>
Closes#17051 from lins05/fix-cbo-filter-file-encoding.
## What changes were proposed in this pull request?
This PR adds tests hive-hash by comparing the outputs generated against Hive 1.2.1. Following datatypes are covered by this PR:
- null
- boolean
- byte
- short
- int
- long
- float
- double
- string
- array
- map
- struct
Datatypes that I have _NOT_ covered but I will work on separately are:
- Decimal (handled separately in https://github.com/apache/spark/pull/17056)
- TimestampType
- DateType
- CalendarIntervalType
## How was this patch tested?
NA
Author: Tejas Patil <tejasp@fb.com>
Closes#17049 from tejasapatil/SPARK-17495_remaining_types.
## What changes were proposed in this pull request?
We traverse predicate and evaluate the logical expressions to compute the selectivity of a FILTER operator.
## How was this patch tested?
We add a new test suite to test various logical operators.
Author: Ron Hu <ron.hu@huawei.com>
Closes#16395 from ron8hu/filterSelectivity.
## What changes were proposed in this pull request?
This PR adds a special streaming deduplication operator to support `dropDuplicates` with `aggregation` and watermark. It reuses the `dropDuplicates` API but creates new logical plan `Deduplication` and new physical plan `DeduplicationExec`.
The following cases are supported:
- one or multiple `dropDuplicates()` without aggregation (with or without watermark)
- `dropDuplicates` before aggregation
Not supported cases:
- `dropDuplicates` after aggregation
Breaking changes:
- `dropDuplicates` without aggregation doesn't work with `complete` or `update` mode.
## How was this patch tested?
The new unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16970 from zsxwing/dedup.
## What changes were proposed in this pull request?
This PR is a small follow-up on https://github.com/apache/spark/pull/16804. This PR also adds support for nested char/varchar fields in orc.
## How was this patch tested?
I have added a regression test to the OrcSourceSuite.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#17030 from hvanhovell/SPARK-19459-follow-up.
## What changes were proposed in this pull request?
This pr fixed a class-cast exception below;
```
scala> spark.range(10).selectExpr("cast (id as decimal) as x").selectExpr("percentile(x, 0.5)").collect()
java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to java.lang.Number
at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:141)
at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:58)
at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.update(interfaces.scala:514)
at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:187)
at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:181)
at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:151)
at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.<init>(ObjectAggregationIterator.scala:78)
at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:109)
at
```
This fix simply converts catalyst values (i.e., `Decimal`) into scala ones by using `CatalystTypeConverters`.
## How was this patch tested?
Added a test in `DataFrameSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17028 from maropu/SPARK-19691.
## What changes were proposed in this pull request?
This pr comes from #16928 and fixed a json behaviour along with the CSV one.
## How was this patch tested?
Added tests in `JsonSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#17023 from maropu/SPARK-19695.
### What changes were proposed in this pull request?
Currently, if `NumPartitions` is not set in RepartitionByExpression, we will set it using `spark.sql.shuffle.partitions` during Planner. However, this is not following the general resolution process. This PR is to set it in `Parser` and then `Optimizer` can use the value for plan optimization.
### How was this patch tested?
Added a test case.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#16988 from gatorsmile/resolveRepartition.
## What changes were proposed in this pull request?
This PR proposes to fix two.
**Skip a property without a getter in beans**
Currently, if we use a JavaBean without the getter as below:
```java
public static class BeanWithoutGetter implements Serializable {
private String a;
public void setA(String a) {
this.a = a;
}
}
BeanWithoutGetter bean = new BeanWithoutGetter();
List<BeanWithoutGetter> data = Arrays.asList(bean);
spark.createDataFrame(data, BeanWithoutGetter.class).show();
```
- Before
It throws an exception as below:
```
java.lang.NullPointerException
at org.spark_project.guava.reflect.TypeToken.method(TypeToken.java:465)
at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:126)
at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:125)
```
- After
```
++
||
++
||
++
```
**Supports empty bean in encoder creation**
```java
public static class EmptyBean implements Serializable {}
EmptyBean bean = new EmptyBean();
List<EmptyBean> data = Arrays.asList(bean);
spark.createDataset(data, Encoders.bean(EmptyBean.class)).show();
```
- Before
throws an exception as below:
```
java.lang.UnsupportedOperationException: Cannot infer type for class EmptyBean because it is not bean-compliant
at org.apache.spark.sql.catalyst.JavaTypeInference$.org$apache$spark$sql$catalyst$JavaTypeInference$$serializerFor(JavaTypeInference.scala:436)
at org.apache.spark.sql.catalyst.JavaTypeInference$.serializerFor(JavaTypeInference.scala:341)
```
- After
```
++
||
++
||
++
```
## How was this patch tested?
Unit test in `JavaDataFrameSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17013 from HyukjinKwon/SPARK-19666.
## What changes were proposed in this pull request?
This is a small change to make GeneratorOuter always unresolved. It is mostly no-op change but makes it more clear since GeneratorOuter shouldn't survive analysis phase.
This requires also handling in ResolveAliases rule.
## How was this patch tested?
Existing generator tests.
Author: Bogdan Raducanu <bogdan@databricks.com>
Author: Reynold Xin <rxin@databricks.com>
Closes#17026 from bogdanrdc/PR16958.
## What changes were proposed in this pull request?
The Range was modified to produce "recordsRead" metric instead of "generated rows". The tests were updated and partially moved to SQLMetricsSuite.
## How was this patch tested?
Unit tests.
Author: Ala Luszczak <ala@databricks.com>
Closes#16960 from ala/range-records-read.
## What changes were proposed in this pull request?
If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory.
Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired.
These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing.
I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one.
## How was this patch tested?
New and existing unit tests. No performance or load tests have been run.
Author: Nathan Howell <nhowell@godaddy.com>
Closes#16386 from NathanHowell/SPARK-18352.
- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings
For the future:
- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#16871 from srowen/SPARK-19493.
## What changes were proposed in this pull request?
Jira: https://issues.apache.org/jira/browse/SPARK-19618
Moved the check for validating number of buckets from `DataFrameWriter` to `BucketSpec` creation
## How was this patch tested?
- Added more unit tests
Author: Tejas Patil <tejasp@fb.com>
Closes#16948 from tejasapatil/SPARK-19618_max_buckets.
## What changes were proposed in this pull request?
This is a follow-up pr of #16308.
This pr enables timezone support in CSV/JSON parsing.
We should introduce `timeZone` option for CSV/JSON datasources (the default value of the option is session local timezone).
The datasources should use the `timeZone` option to format/parse to write/read timestamp values.
Notice that while reading, if the timestampFormat has the timezone info, the timezone will not be used because we should respect the timezone in the values.
For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values written with the default timezone option, which is `"GMT"` because session local timezone is `"GMT"` here, are:
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")
scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]
scala> df.show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
scala> df.write.json("/path/to/gmtjson")
```
```sh
$ cat /path/to/gmtjson/part-*
{"ts":"2016-01-01T00:00:00.000Z"}
```
whereas setting the option to `"PST"`, they are:
```scala
scala> df.write.option("timeZone", "PST").json("/path/to/pstjson")
```
```sh
$ cat /path/to/pstjson/part-*
{"ts":"2015-12-31T16:00:00.000-08:00"}
```
We can properly read these files even if the timezone option is wrong because the timestamp values have timezone info:
```scala
scala> val schema = new StructType().add("ts", TimestampType)
schema: org.apache.spark.sql.types.StructType = StructType(StructField(ts,TimestampType,true))
scala> spark.read.schema(schema).json("/path/to/gmtjson").show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
scala> spark.read.schema(schema).option("timeZone", "PST").json("/path/to/gmtjson").show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```
And even if `timezoneFormat` doesn't contain timezone info, we can properly read the values with setting correct timezone option:
```scala
scala> df.write.option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson")
```
```sh
$ cat /path/to/jstjson/part-*
{"ts":"2016-01-01T09:00:00"}
```
```scala
// wrong result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").json("/path/to/jstjson").show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 09:00:00|
+-------------------+
// correct result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson").show()
+-------------------+
|ts |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```
This pr also makes `JsonToStruct` and `StructToJson` `TimeZoneAwareExpression` to be able to evaluate values with timezone option.
## How was this patch tested?
Existing tests and added some tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#16750 from ueshin/issues/SPARK-18937.
Move `SQLViewSuite` from `sql/hive` to `sql/core`, so we can test the view supports without hive metastore. Also moved the test cases that specified to hive to `HiveSQLViewSuite`.
Improve the test coverage of SQLViewSuite, cover the following cases:
1. view resolution(possibly a referenced table/view have changed after the view creation);
2. handle a view with user specified column names;
3. improve the test cases for a nested view.
Also added a test case for cyclic view reference, which is a known issue that is not fixed yet.
N/A
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16674 from jiangxb1987/view-test.
## What changes were proposed in this pull request?
A follow-up to disallow space as the delimiter in broadcast hint.
## How was this patch tested?
Jenkins test.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#16941 from viirya/disallow-space-delimiter.
## What changes were proposed in this pull request?
Support cardinality estimation and stats propagation for all join types.
Limitations:
- For inner/outer joins without any equal condition, we estimate it like cartesian product.
- For left semi/anti joins, since we can't apply the heuristics for inner join to it, for now we just propagate the statistics from left side. We should support them when other advanced stats (e.g. histograms) are available in spark.
## How was this patch tested?
Add a new test suite.
Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#16228 from wzhfy/joinEstimate.
## What changes were proposed in this pull request?
A small update to https://github.com/apache/spark/pull/16925
1. Rename SubstituteHints -> ResolveHints to be more consistent with rest of the rules.
2. Added more documentation in the rule and be more defensive / future proof to skip views as well as CTEs.
## How was this patch tested?
This pull request contains no real logic change and all behavior should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#16939 from rxin/SPARK-16475.
## What changes were proposed in this pull request?
The reason for test failure is that the property “oracle.jdbc.mapDateToTimestamp” set by the test was getting converted into all lower case. Oracle database expects this property in case-sensitive manner.
This test was passing in previous releases because connection properties were sent as user specified for the test case scenario. Fixes to handle all option uniformly in case-insensitive manner, converted the JDBC connection properties also to lower case.
This PR enhances CaseInsensitiveMap to keep track of input case-sensitive keys , and uses those when creating connection properties that are passed to the JDBC connection.
Alternative approach PR https://github.com/apache/spark/pull/16847 is to pass original input keys to JDBC data source by adding check in the Data source class and handle case-insensitivity in the JDBC source code.
## How was this patch tested?
Added new test cases to JdbcSuite , and OracleIntegrationSuite. Ran docker integration tests passed on my laptop, all tests passed successfully.
Author: sureshthalamati <suresh.thalamati@gmail.com>
Closes#16891 from sureshthalamati/jdbc_case_senstivity_props_fix-SPARK-19318.
## What changes were proposed in this pull request?
This pull request introduces a simple hint infrastructure to SQL and implements broadcast join hint using the infrastructure.
The hint syntax looks like the following:
```
SELECT /*+ BROADCAST(t) */ * FROM t
```
For broadcast hint, we accept "BROADCAST", "BROADCASTJOIN", and "MAPJOIN", and a sequence of relation aliases can be specified in the hint. A broadcast hint plan node will be inserted on top of any relation (that is not aliased differently), subquery, or common table expression that match the specified name.
The hint resolution works by recursively traversing down the query plan to find a relation or subquery that matches one of the specified broadcast aliases. The traversal does not go past beyond any existing broadcast hints, subquery aliases. This rule happens before common table expressions.
Note that there was an earlier patch in https://github.com/apache/spark/pull/14426. This is a rewrite of that patch, with different semantics and simpler test cases.
## How was this patch tested?
Added a new unit test suite for the broadcast hint rule (SubstituteHintsSuite) and new test cases for parser change (in PlanParserSuite). Also added end-to-end test case in BroadcastSuite.
Author: Reynold Xin <rxin@databricks.com>
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#16925 from rxin/SPARK-16475-broadcast-hint.
What changes were proposed in this pull request?
Support CREATE [EXTERNAL] TABLE LIKE LOCATION... syntax for Hive serde and datasource tables.
In this PR,we follow SparkSQL design rules :
supporting create table like view or physical table or temporary view with location.
creating a table with location,this table will be an external table other than managed table.
How was this patch tested?
Add new test cases and update existing test cases
Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn>
Closes#16868 from ouyangxiaochen/spark19115.
## What changes were proposed in this pull request?
This PR proposes to support type coercion between `ArrayType`s where the element types are compatible.
**Before**
```
Seq(Array(1)).toDF("a").selectExpr("greatest(a, array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(`a`, array(1.0D))' due to data type mismatch: The expressions should all have the same type, got GREATEST(array<int>, array<double>).; line 1 pos 0;
Seq(Array(1)).toDF("a").selectExpr("least(a, array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve 'least(`a`, array(1.0D))' due to data type mismatch: The expressions should all have the same type, got LEAST(array<int>, array<double>).; line 1 pos 0;
sql("SELECT * FROM values (array(0)), (array(1D)) as data(a)")
org.apache.spark.sql.AnalysisException: incompatible types found in column a for inline table; line 1 pos 14
Seq(Array(1)).toDF("a").union(Seq(Array(1D)).toDF("b"))
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. ArrayType(DoubleType,false) <> ArrayType(IntegerType,false) at the first column of the second table;;
sql("SELECT IF(1=1, array(1), array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve '(IF((1 = 1), array(1), array(1.0D)))' due to data type mismatch: differing types in '(IF((1 = 1), array(1), array(1.0D)))' (array<int> and array<double>).; line 1 pos 7;
```
**After**
```scala
Seq(Array(1)).toDF("a").selectExpr("greatest(a, array(1D))")
res5: org.apache.spark.sql.DataFrame = [greatest(a, array(1.0)): array<double>]
Seq(Array(1)).toDF("a").selectExpr("least(a, array(1D))")
res6: org.apache.spark.sql.DataFrame = [least(a, array(1.0)): array<double>]
sql("SELECT * FROM values (array(0)), (array(1D)) as data(a)")
res8: org.apache.spark.sql.DataFrame = [a: array<double>]
Seq(Array(1)).toDF("a").union(Seq(Array(1D)).toDF("b"))
res10: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [a: array<double>]
sql("SELECT IF(1=1, array(1), array(1D))")
res15: org.apache.spark.sql.DataFrame = [(IF((1 = 1), array(1), array(1.0))): array<double>]
```
## How was this patch tested?
Unit tests in `TypeCoercion` and Jenkins tests and
building with scala 2.10
```scala
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16777 from HyukjinKwon/SPARK-19435.
## What changes were proposed in this pull request?
This PR proposes to fix the error message when some data types are compatible and others are not in set/union operation.
Currently, the code below:
```scala
Seq((1,("a", 1))).toDF.union(Seq((1L,("a", "b"))).toDF)
```
throws an exception saying `LongType` and `IntegerType` are incompatible types. It should say something about `StructType`s with more readable format as below:
**Before**
```
Union can only be performed on tables with the compatible column types.
LongType <> IntegerType at the first column of the second table;;
```
**After**
```
Union can only be performed on tables with the compatible column types.
struct<_1:string,_2:string> <> struct<_1:string,_2:int> at the second column of the second table;;
```
*I manually inserted a newline in the messages above for readability only in this PR description.
## How was this patch tested?
Unit tests in `AnalysisErrorSuite`, manual tests and build wth Scala 2.10.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16882 from HyukjinKwon/SPARK-19544.
## What changes were proposed in this pull request?
Currently the udf `to_date` has different return value with an invalid date input.
```
SELECT to_date('2015-07-22', 'yyyy-dd-MM') -> return `2016-10-07`
SELECT to_date('2014-31-12') -> return null
```
As discussed in JIRA [SPARK-19496](https://issues.apache.org/jira/browse/SPARK-19496), we should return null in both situations when the input date is invalid
## How was this patch tested?
unit test added
Author: windpiger <songjun@outlook.com>
Closes#16870 from windpiger/to_date.
## What changes were proposed in this pull request?
Reading from an existing ORC table which contains `char` or `varchar` columns can fail with a `ClassCastException` if the table metadata has been created using Spark. This is caused by the fact that spark internally replaces `char` and `varchar` columns with a `string` column.
This PR fixes this by adding the hive type to the `StructField's` metadata under the `HIVE_TYPE_STRING` key. This is picked up by the `HiveClient` and the ORC reader, see https://github.com/apache/spark/pull/16060 for more details on how the metadata is used.
## How was this patch tested?
Added a regression test to `OrcSourceSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16804 from hvanhovell/SPARK-19459.
## What changes were proposed in this pull request?
Using from_json on a column with an empty string results in: java.util.NoSuchElementException: head of empty list.
This is because `parser.parse(input)` may return `Nil` when `input.trim.isEmpty`
## How was this patch tested?
Regression test in `JsonExpressionsSuite`
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#16881 from brkyvz/json-fix.
## What changes were proposed in this pull request?
With the new approach of view resolution, we can get rid of SQL generation on view creation, so let's remove SQL builder for operators.
Note that, since all sql generation for operators is defined in one file (org.apache.spark.sql.catalyst.SQLBuilder), it’d be trivial to recover it in the future.
## How was this patch tested?
N/A
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16869 from jiangxb1987/SQLBuilder.
## What changes were proposed in this pull request?
Set currentVars to null in GenerateOrdering.genComparisons before genCode is called. genCode ignores INPUT_ROW if currentVars is not null and in genComparisons we want it to use INPUT_ROW.
## How was this patch tested?
Added test with 2 queries in WholeStageCodegenSuite
Author: Bogdan Raducanu <bogdan.rdc@gmail.com>
Closes#16852 from bogdanrdc/SPARK-19512.
## What changes were proposed in this pull request?
Previously range operator could not be interrupted. For example, using DAGScheduler.cancelStage(...) on a query with range might have been ineffective.
This change adds periodic checks of TaskContext.isInterrupted to codegen version, and InterruptibleOperator to non-codegen version.
I benchmarked the performance of codegen version on a sample query `spark.range(1000L * 1000 * 1000 * 10).count()` and there is no measurable difference.
## How was this patch tested?
Adds a unit test.
Author: Ala Luszczak <ala@databricks.com>
Closes#16872 from ala/SPARK-19514b.
## What changes were proposed in this pull request?
SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable.
In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`.
## How was this patch tested?
new test case: `test("conf entry: checkValue()")`
Author: Liwei Lin <lwlin7@gmail.com>
Closes#16736 from lw-lin/conf.
## What changes were proposed in this pull request?
Hive metastore is not case-preserving and keep partition columns with lower case names. If Spark SQL creates a table with upper-case partition column names using `HiveExternalCatalog`, when we rename partition, it first calls the HiveClient to renamePartition, which will create a new lower case partition path, then Spark SQL renames the lower case path to upper-case.
However, when we rename a nested path, different file systems have different behaviors. e.g. in jenkins, renaming `a=1/b=2` to `A=2/B=2` will success, but leave an empty directory `a=1`. in mac os, the renaming doesn't work as expected and result to `a=1/B=2`.
This PR renames the partition directory recursively from the first partition column in `HiveExternalCatalog`, to be most compatible with different file systems.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16837 from cloud-fan/partition.
### What changes were proposed in this pull request?
`table.schema` is always not empty for partitioned tables, because `table.schema` also contains the partitioned columns, even if the original table does not have any column. This PR is to fix the issue.
### How was this patch tested?
Added a test case
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16848 from gatorsmile/inferHiveSerdeSchema.
## What changes were proposed in this pull request?
`mapGroupsWithState` is a new API for arbitrary stateful operations in Structured Streaming, similar to `DStream.mapWithState`
*Requirements*
- Users should be able to specify a function that can do the following
- Access the input row corresponding to a key
- Access the previous state corresponding to a key
- Optionally, update or remove the state
- Output any number of new rows (or none at all)
*Proposed API*
```
// ------------ New methods on KeyValueGroupedDataset ------------
class KeyValueGroupedDataset[K, V] {
// Scala friendly
def mapGroupsWithState[S: Encoder, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => U)
def flatMapGroupsWithState[S: Encode, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => Iterator[U])
// Java friendly
def mapGroupsWithState[S, U](func: MapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
def flatMapGroupsWithState[S, U](func: FlatMapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
}
// ------------------- New Java-friendly function classes -------------------
public interface MapGroupsWithStateFunction<K, V, S, R> extends Serializable {
R call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}
public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends Serializable {
Iterator<R> call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}
// ---------------------- Wrapper class for state data ----------------------
trait State[S] {
def exists(): Boolean
def get(): S // throws Exception is state does not exist
def getOption(): Option[S]
def update(newState: S): Unit
def remove(): Unit // exists() will be false after this
}
```
Key Semantics of the State class
- The state can be null.
- If the state.remove() is called, then state.exists() will return false, and getOption will returm None.
- After that state.update(newState) is called, then state.exists() will return true, and getOption will return Some(...).
- None of the operations are thread-safe. This is to avoid memory barriers.
*Usage*
```
val stateFunc = (word: String, words: Iterator[String, runningCount: KeyedState[Long]) => {
val newCount = words.size + runningCount.getOption.getOrElse(0L)
runningCount.update(newCount)
(word, newCount)
}
dataset // type is Dataset[String]
.groupByKey[String](w => w) // generates KeyValueGroupedDataset[String, String]
.mapGroupsWithState[Long, (String, Long)](stateFunc) // returns Dataset[(String, Long)]
```
## How was this patch tested?
New unit tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#16758 from tdas/mapWithState.
## What changes were proposed in this pull request?
The optimizer tries to remove redundant alias only projections from the query plan using the `RemoveAliasOnlyProject` rule. The current rule identifies removes such a project and rewrites the project's attributes in the **entire** tree. This causes problems when parts of the tree are duplicated (for instance a self join on a temporary view/CTE) and the duplicated part contains the alias only project, in this case the rewrite will break the tree.
This PR fixes these problems by using a blacklist for attributes that are not to be moved, and by making sure that attribute remapping is only done for the parent tree, and not for unrelated parts of the query plan.
The current tree transformation infrastructure works very well if the transformation at hand requires little or a global contextual information. In this case we need to know both the attributes that were not to be moved, and we also needed to know which child attributes were modified. This cannot be done easily using the current infrastructure, and solutions typically involves transversing the query plan multiple times (which is super slow). I have moved around some code in `TreeNode`, `QueryPlan` and `LogicalPlan`to make this much more straightforward; this basically allows you to manually traverse the tree.
This PR subsumes the following PRs by windpiger:
Closes https://github.com/apache/spark/pull/16267
Closes https://github.com/apache/spark/pull/16255
## How was this patch tested?
I have added unit tests to `RemoveRedundantAliasAndProjectSuite` and I have added integration tests to the `SQLQueryTestSuite.union` and `SQLQueryTestSuite.cte` test cases.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16757 from hvanhovell/SPARK-18609.
## What changes were proposed in this pull request?
This pull request adds two new user facing functions:
- `to_date` which accepts an expression and a format and returns a date.
- `to_timestamp` which accepts an expression and a format and returns a timestamp.
For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM)
### Date Function
*Previously*
```
to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp"))
```
*Current*
```
to_date(lit("2016-21-05"), "yyyy-dd-MM")
```
### Timestamp Function
*Previously*
```
unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")
```
*Current*
```
to_timestamp(lit("2016-21-05"), "yyyy-dd-MM")
```
### Tasks
- [X] Add `to_date` to Scala Functions
- [x] Add `to_date` to Python Functions
- [x] Add `to_date` to SQL Functions
- [X] Add `to_timestamp` to Scala Functions
- [x] Add `to_timestamp` to Python Functions
- [x] Add `to_timestamp` to SQL Functions
- [x] Add function to R
## How was this patch tested?
- [x] Add Functions to `DateFunctionsSuite`
- Test new `ParseToTimestamp` Expression (*not necessary*)
- Test new `ParseToDate` Expression (*not necessary*)
- [x] Add test for R
- [x] Add test for Python in test.py
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: anabranch <wac.chambers@gmail.com>
Author: Bill Chambers <bill@databricks.com>
Author: anabranch <bill@databricks.com>
Closes#16138 from anabranch/SPARK-16609.
## What changes were proposed in this pull request?
I have a frequency distribution table with following entries
Age, No of person
21, 10
22, 15
23, 18
..
..
30, 14
Moreover it is common to have data in frequency distribution format to further calculate Percentile, Median. With current implementation
It would be very difficult and complex to find the percentile.
Therefore i am proposing enhancement to current Percentile and Approx Percentile implementation to take frequency distribution column into consideration
## How was this patch tested?
1) Enhanced /sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala to cover the additional functionality
2) Run some performance benchmark test with 20 million row in local environment and did not see any performance degradation
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: gagan taneja <tanejagagan@gagans-MacBook-Pro.local>
Closes#16497 from tanejagagan/branch-18940.
## What changes were proposed in this pull request?
It often happens that a complex object (struct/map/array) is created only to get elements from it in an subsequent expression. We can add an optimizer rule for this.
## How was this patch tested?
unit-tests
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Eyal Farago <eyal@nrgene.com>
Author: eyal farago <eyal.farago@gmail.com>
Closes#16043 from eyalfa/SPARK-18601.
### What changes were proposed in this pull request?
The removed codes for `IN` are not reachable, because the previous rule `InConversion` already resolves the type coercion issues.
### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16783 from gatorsmile/typeCoercionIn.
## What changes were proposed in this pull request?
The SQL parser can mistake a `WHEN (...)` used in `CASE` for a function call. This happens in cases like the following:
```sql
select case when (1) + case when 1 > 0 then 1 else 0 end = 2 then 1 else 0 end
from tb
```
This PR fixes this by re-organizing the case related parsing rules.
## How was this patch tested?
Added a regression test to the `ExpressionParserSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16821 from hvanhovell/SPARK-19472.
## What changes were proposed in this pull request?
The current way of resolving `InsertIntoTable` and `CreateTable` is convoluted: sometimes we replace them with concrete implementation commands during analysis, sometimes during planning phase.
And the error checking logic is also a mess: we may put it in extended analyzer rules, or extended checking rules, or `CheckAnalysis`.
This PR simplifies the data source analysis:
1. `InsertIntoTable` and `CreateTable` are always unresolved and need to be replaced by concrete implementation commands during analysis.
2. The error checking logic is mainly in 2 rules: `PreprocessTableCreation` and `PreprocessTableInsertion`.
## How was this patch tested?
existing test.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16269 from cloud-fan/ddl.
## What changes were proposed in this pull request?
DataFrame.except doesn't work for UDT columns. It is because `ExtractEquiJoinKeys` will run `Literal.default` against UDT. However, we don't handle UDT in `Literal.default` and an exception will throw like:
java.lang.RuntimeException: no default for type
org.apache.spark.ml.linalg.VectorUDT3bfc3ba7
at org.apache.spark.sql.catalyst.expressions.Literal$.default(literals.scala:179)
at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:117)
at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:110)
More simple fix is just let `Literal.default` handle UDT by its sql type. So we can use more efficient join type on UDT.
Besides `except`, this also fixes other similar scenarios, so in summary this fixes:
* `except` on two Datasets with UDT
* `intersect` on two Datasets with UDT
* `Join` with the join conditions using `<=>` on UDT columns
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#16765 from viirya/df-except-for-udt.
## What changes were proposed in this pull request?
This PR proposes to
- remove unused `findTightestCommonType` in `TypeCoercion` as suggested in https://github.com/apache/spark/pull/16777#discussion_r99283834
- rename `findTightestCommonTypeOfTwo ` to `findTightestCommonType`.
- fix comments accordingly
The usage was removed while refactoring/fixing in several JIRAs such as SPARK-16714, SPARK-16735 and SPARK-16646
## How was this patch tested?
Existing tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16786 from HyukjinKwon/SPARK-19446.
## What changes were proposed in this pull request?
In `ExpressionEncoder.toRow` and `fromRow`, we catch the exception and output `treeString` of serializer/deserializer expressions in the error message. However, encoder can be very complex and the serializer/deserializer expressions can be very large trees and blow up the log files(e.g. generate over 500mb logs for this single error message.) As a first attempt, this PR try to use `simpleString` instead.
**BEFORE**
```scala
scala> :paste
// Entering paste mode (ctrl-D to finish)
case class TestCaseClass(value: Int)
import spark.implicits._
Seq(TestCaseClass(1)).toDS().collect()
// Exiting paste mode, now interpreting.
java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
newInstance(class TestCaseClass)
+- assertnotnull(input[0, int, false], - field (class: "scala.Int", name: "value"), - root class: "TestCaseClass")
+- input[0, int, false]
at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:303)
...
```
**AFTER**
```scala
...
// Exiting paste mode, now interpreting.
java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
newInstance(class TestCaseClass)
at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:303)
...
```
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#16701 from dongjoon-hyun/SPARK-18909-EXPR-ERROR.
## What changes were proposed in this pull request?
There is a metadata introduced before to mark the optional columns in merged Parquet schema for filter predicate pushdown. As we upgrade to Parquet 1.8.2 which includes the fix for the pushdown of optional columns, we don't need this metadata now.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#16756 from viirya/remove-optional-metadata.
## What changes were proposed in this pull request?
This PR proposes three things as below:
- Support LaTex inline-formula, `\( ... \)` in Scala API documentation
It seems currently,
```
\( ... \)
```
are rendered as they are, for example,
<img width="345" alt="2017-01-30 10 01 13" src="https://cloud.githubusercontent.com/assets/6477701/22423960/ab37d54a-e737-11e6-9196-4f6229c0189c.png">
It seems mistakenly more backslashes were added.
- Fix warnings Scaladoc/Javadoc generation
This PR fixes t two types of warnings as below:
```
[warn] .../spark/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala:335: Could not find any member to link for "UnsupportedOperationException".
[warn] /**
[warn] ^
```
```
[warn] .../spark/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala:24: Variable var undefined in comment for class VariableSubstitution in class VariableSubstitution
[warn] * `${var}`, `${system:var}` and `${env:var}`.
[warn] ^
```
- Fix Javadoc8 break
```
[error] .../spark/mllib/target/java/org/apache/spark/ml/PredictionModel.java:7: error: reference not found
[error] * E.g., {link VectorUDT} for vector features.
[error] ^
[error] .../spark/mllib/target/java/org/apache/spark/ml/PredictorParams.java:12: error: reference not found
[error] * E.g., {link VectorUDT} for vector features.
[error] ^
[error] .../spark/mllib/target/java/org/apache/spark/ml/Predictor.java:10: error: reference not found
[error] * E.g., {link VectorUDT} for vector features.
[error] ^
[error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/HiveAnalysis.java:5: error: reference not found
[error] * Note that, this rule must be run after {link PreprocessTableInsertion}.
[error] ^
```
## How was this patch tested?
Manually via `sbt unidoc` and `jeykil build`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16741 from HyukjinKwon/warn-and-break.
### What changes were proposed in this pull request?
Currently, the function `to_json` allows users to provide options for generating JSON. However, it does not pass it to `JacksonGenerator`. Thus, it ignores the user-provided options. This PR is to fix it. Below is an example.
```Scala
val df = Seq(Tuple1(Tuple1(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))).toDF("a")
val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm")
df.select(to_json($"a", options)).show(false)
```
The current output is like
```
+--------------------------------------+
|structtojson(a) |
+--------------------------------------+
|{"_1":"2015-08-26T18:00:00.000-07:00"}|
+--------------------------------------+
```
After the fix, the output is like
```
+-------------------------+
|structtojson(a) |
+-------------------------+
|{"_1":"26/08/2015 18:00"}|
+-------------------------+
```
### How was this patch tested?
Added test cases for both `from_json` and `to_json`
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16745 from gatorsmile/toJson.
## What changes were proposed in this pull request?
During canonicalization, `NOT(...(l, r))` should not expect such cases that `l.hashcode > r.hashcode`.
Take the rule `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` for example, it should never be matched since `GreaterThan(l, r)` itself would be re-written as `GreaterThan(r, l)` given `l.hashcode > r.hashcode` after canonicalization.
This patch consolidates rules like `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` and `case NOT(GreaterThan(l, r))`.
## How was this patch tested?
This patch expanded the `NOT` test case to cover both cases where:
- `l.hashcode > r.hashcode`
- `l.hashcode < r.hashcode`
Author: Liwei Lin <lwlin7@gmail.com>
Closes#16719 from lw-lin/canonicalize.
## What changes were proposed in this pull request?
This PR fixes both,
javadoc8 break
```
[error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/FindHiveSerdeTable.java:3: error: reference not found
[error] * Replaces {link SimpleCatalogRelation} with {link MetastoreRelation} if its table provider is hive.
```
and the example in `StructType` as a self-contained example as below:
```scala
import org.apache.spark.sql._
import org.apache.spark.sql.types._
val struct =
StructType(
StructField("a", IntegerType, true) ::
StructField("b", LongType, false) ::
StructField("c", BooleanType, false) :: Nil)
// Extract a single StructField.
val singleField = struct("b")
// singleField: StructField = StructField(b,LongType,false)
// If this struct does not have a field called "d", it throws an exception.
struct("d")
// java.lang.IllegalArgumentException: Field "d" does not exist.
// ...
// Extract multiple StructFields. Field names are provided in a set.
// A StructType object will be returned.
val twoFields = struct(Set("b", "c"))
// twoFields: StructType =
// StructType(StructField(b,LongType,false), StructField(c,BooleanType,false))
// Any names without matching fields will throw an exception.
// For the case shown below, an exception is thrown due to "d".
struct(Set("b", "c", "d"))
// java.lang.IllegalArgumentException: Field "d" does not exist.
// ...
```
```scala
import org.apache.spark.sql._
import org.apache.spark.sql.types._
val innerStruct =
StructType(
StructField("f1", IntegerType, true) ::
StructField("f2", LongType, false) ::
StructField("f3", BooleanType, false) :: Nil)
val struct = StructType(
StructField("a", innerStruct, true) :: Nil)
// Create a Row with the schema defined by struct
val row = Row(Row(1, 2, true))
```
Also, now when the column is missing, it throws an exception rather than ignoring.
## How was this patch tested?
Manually via `sbt unidoc`.
- Scaladoc
<img width="665" alt="2017-01-26 12 54 13" src="https://cloud.githubusercontent.com/assets/6477701/22297905/1245620e-e362-11e6-9e22-43bb8d9871af.png">
- Javadoc
<img width="722" alt="2017-01-26 12 54 27" src="https://cloud.githubusercontent.com/assets/6477701/22297899/0fd87e0c-e362-11e6-9033-7590bda1aea6.png">
<img width="702" alt="2017-01-26 12 54 32" src="https://cloud.githubusercontent.com/assets/6477701/22297900/0fe14154-e362-11e6-9882-768381c53163.png">
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16703 from HyukjinKwon/SPARK-12970.
## What changes were proposed in this pull request?
This pr added a variable for a UDF name in `ScalaUDF`.
Then, if the variable filled, `DataFrame#explain` prints the name.
## How was this patch tested?
Added a test in `UDFSuite`.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#16707 from maropu/SPARK-19338.
## What changes were proposed in this pull request?
As of Spark 2.1, Spark SQL assumes the machine timezone for datetime manipulation, which is bad if users are not in the same timezones as the machines, or if different users have different timezones.
We should introduce a session local timezone setting that is used for execution.
An explicit non-goal is locale handling.
### Semantics
Setting the session local timezone means that the timezone-aware expressions listed below should use the timezone to evaluate values, and also it should be used to convert (cast) between string and timestamp or between timestamp and date.
- `CurrentDate`
- `CurrentBatchTimestamp`
- `Hour`
- `Minute`
- `Second`
- `DateFormatClass`
- `ToUnixTimestamp`
- `UnixTimestamp`
- `FromUnixTime`
and below are implicitly timezone-aware through cast from timestamp to date:
- `DayOfYear`
- `Year`
- `Quarter`
- `Month`
- `DayOfMonth`
- `WeekOfYear`
- `LastDay`
- `NextDay`
- `TruncDate`
For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values evaluated by some of timezone-aware expressions are:
```scala
scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]
scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2016-01-01 00:00:00|2016 |1 |1 |0 |0 |0 |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```
whereas setting the session local timezone to `"PST"`, they are:
```scala
scala> spark.conf.set("spark.sql.session.timeZone", "PST")
scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2015-12-31 16:00:00|2015 |12 |31 |16 |0 |0 |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```
Notice that even if you set the session local timezone, it affects only in `DataFrame` operations, neither in `Dataset` operations, `RDD` operations nor in `ScalaUDF`s. You need to properly handle timezone by yourself.
### Design of the fix
I introduced an analyzer to pass session local timezone to timezone-aware expressions and modified DateTimeUtils to take the timezone argument.
## How was this patch tested?
Existing tests and added tests for timezone aware expressions.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#16308 from ueshin/issues/SPARK-18350.
## What changes were proposed in this pull request?
acceptType() in UDT will no only accept the same type but also all base types
## How was this patch tested?
Manual test using a set of generated UDTs fixing acceptType() in my user defined types
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: gmoehler <moehler@de.ibm.com>
Closes#16660 from gmoehler/master.
## What changes were proposed in this pull request?
This PR will report proper error messages when a subquery expression contain an invalid plan. This problem is fixed by calling CheckAnalysis for the plan inside a subquery.
## How was this patch tested?
Existing tests and two new test cases on 2 forms of subquery, namely, scalar subquery and in/exists subquery.
````
-- TC 01.01
-- The column t2b in the SELECT of the subquery is invalid
-- because it is neither an aggregate function nor a GROUP BY column.
select t1a, t2b
from t1, t2
where t1b = t2c
and t2b = (select max(avg)
from (select t2b, avg(t2b) avg
from t2
where t2a = t1.t1b
)
)
;
-- TC 01.02
-- Invalid due to the column t2b not part of the output from table t2.
select *
from t1
where t1a in (select min(t2a)
from t2
group by t2c
having t2c in (select max(t3c)
from t3
group by t3b
having t3b > t2b ))
;
````
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#16572 from nsyca/18863.
## What changes were proposed in this pull request?
This PR fixes the code in Optimizer phase where the NULL-aware expression of a NOT IN query is expanded in Rule `RewritePredicateSubquery`.
Example:
The query
select a1,b1
from t1
where (a1,b1) not in (select a2,b2
from t2);
has the (a1, b1) = (a2, b2) rewritten from (before this fix):
Join LeftAnti, ((isnull((_1#2 = a2#16)) || isnull((_2#3 = b2#17))) || ((_1#2 = a2#16) && (_2#3 = b2#17)))
to (after this fix):
Join LeftAnti, (((_1#2 = a2#16) || isnull((_1#2 = a2#16))) && ((_2#3 = b2#17) || isnull((_2#3 = b2#17))))
## How was this patch tested?
sql/test, catalyst/test and new test cases in SQLQueryTestSuite.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#16467 from nsyca/19017.
## What changes were proposed in this pull request?
Spark SQL follows MySQL to do the implicit type conversion for binary comparison: http://dev.mysql.com/doc/refman/5.7/en/type-conversion.html
However, this may return confusing result, e.g. `1 = 'true'` will return true, `19157170390056973L = '19157170390056971'` will return true.
I think it's more reasonable to follow postgres in this case, i.e. cast string to the type of the other side, but return null if the string is not castable to keep hive compatibility.
## How was this patch tested?
newly added tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15880 from cloud-fan/compare.
## What changes were proposed in this pull request?
CataLogTable's partitionSchema should check if each column name in partitionColumnNames must match one and only one field in schema, if not we should throw an exception
and CataLogTable's partitionSchema should keep order with partitionColumnNames
## How was this patch tested?
N/A
Author: windpiger <songjun@outlook.com>
Closes#16606 from windpiger/checkPartionColNameWithSchema.
## What changes were proposed in this pull request?
Hive will expand the view text, so it needs 2 fields: originalText and viewText. Since we don't expand the view text, but only add table properties, perhaps only a single field `viewText` is enough in CatalogTable.
This PR brought in the following changes:
1. Remove the param `viewOriginalText` from `CatalogTable`;
2. Update the output of command `DescribeTableCommand`.
## How was this patch tested?
Tested by exsiting test cases, also updated the failed test cases.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16679 from jiangxb1987/catalogTable.
## What changes were proposed in this pull request?
To implement DDL commands, we added several analyzer rules in sql/hive module to analyze DDL related plans. However, our `Analyzer` currently only have one extending interface: `extendedResolutionRules`, which defines extra rules that will be run together with other rules in the resolution batch, and doesn't fit DDL rules well, because:
1. DDL rules may do some checking and normalization, but we may do it many times as the resolution batch will run rules again and again, until fixed point, and it's hard to tell if a DDL rule has already done its checking and normalization. It's fine because DDL rules are idempotent, but it's bad for analysis performance
2. some DDL rules may depend on others, and it's pretty hard to write `if` conditions to guarantee the dependencies. It will be good if we have a batch which run rules in one pass, so that we can guarantee the dependencies by rules order.
This PR adds a new extending interface in `Analyzer`: `postHocResolutionRules`, which defines rules that will be run only once in a batch runs right after the resolution batch.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16645 from cloud-fan/analyzer.
## What changes were proposed in this pull request?
As I pointed out in https://github.com/apache/spark/pull/15807#issuecomment-259143655 , the current subexpression elimination framework has a problem, it always evaluates all common subexpressions at the beginning, even they are inside conditional expressions and may not be accessed.
Ideally we should implement it like scala lazy val, so we only evaluate it when it gets accessed at lease once. https://github.com/apache/spark/issues/15837 tries this approach, but it seems too complicated and may introduce performance regression.
This PR simply stops common subexpression elimination for conditional expressions, with some cleanup.
## How was this patch tested?
regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16659 from cloud-fan/codegen.
### What changes were proposed in this pull request?
It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables.
### How was this patch tested?
Fixed the test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16587 from gatorsmile/blockHiveTable.
## What changes were proposed in this pull request?
PythonUDF is unevaluable, which can not be used inside a join condition, currently the optimizer will push a PythonUDF which accessing both side of join into the join condition, then the query will fail to plan.
This PR fix this issue by checking the expression is evaluable or not before pushing it into Join.
## How was this patch tested?
Add a regression test.
Author: Davies Liu <davies@databricks.com>
Closes#16581 from davies/pyudf_join.
## What changes were proposed in this pull request?
Sort in a streaming plan should be allowed only after a aggregation in complete mode. Currently it is incorrectly allowed when present anywhere in the plan. It gives unpredictable potentially incorrect results.
## How was this patch tested?
New test
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#16662 from tdas/SPARK-19314.
## What changes were proposed in this pull request?
Change non-cbo estimation behavior of aggregate:
- If groupExpression is empty, we can know row count (=1) and the corresponding size;
- otherwise, estimation falls back to UnaryNode's computeStats method, which should not propagate rowCount and attributeStats in Statistics because they are not estimated in that method.
## How was this patch tested?
Added test case
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#16631 from wzhfy/aggNoCbo.
## What changes were proposed in this pull request?
We have a table relation plan cache in `HiveMetastoreCatalog`, which caches a lot of things: file status, resolved data source, inferred schema, etc.
However, it doesn't make sense to limit this cache with hive support, we should move it to SQL core module so that users can use this cache without hive support.
It can also reduce the size of `HiveMetastoreCatalog`, so that it's easier to remove it eventually.
main changes:
1. move the table relation cache to `SessionCatalog`
2. `SessionCatalog.lookupRelation` will return `SimpleCatalogRelation` and the analyzer will convert it to `LogicalRelation` or `MetastoreRelation` later, then `HiveSessionCatalog` doesn't need to override `lookupRelation` anymore
3. `FindDataSourceTable` will read/write the table relation cache.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16621 from cloud-fan/plan-cache.
## What changes were proposed in this pull request?
On CREATE/ALTER a view, it's no longer needed to generate a SQL text string from the LogicalPlan, instead we store the SQL query text、the output column names of the query plan, and current database to CatalogTable. Permanent views created by this approach can be resolved by current view resolution approach.
The main advantage includes:
1. If you update an underlying view, the current view also gets updated;
2. That gives us a change to get ride of SQL generation for operators.
Major changes of this PR:
1. Generate the view-specific properties(e.g. view default database, view query output column names) during permanent view creation and store them as properties in the CatalogTable;
2. Update the commands `CreateViewCommand` and `AlterViewAsCommand`, get rid of SQL generation from them.
## How was this patch tested?
Existing tests.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16613 from jiangxb1987/view-write-path.
## What changes were proposed in this pull request?
remove ununsed imports and outdated comments, and fix some minor code style issue.
## How was this patch tested?
existing ut
Author: uncleGen <hustyugm@gmail.com>
Closes#16591 from uncleGen/SPARK-19227.
## What changes were proposed in this pull request?
Added outer_explode, outer_posexplode, outer_inline functions and expressions.
Some bug fixing in GenerateExec.scala for CollectionGenerator. Previously it was not correctly handling the case of outer with empty collections, only with nulls.
## How was this patch tested?
New tests added to GeneratorFunctionSuite
Author: Bogdan Raducanu <bogdan.rdc@gmail.com>
Closes#16608 from bogdanrdc/SPARK-13721.
## What changes were proposed in this pull request?
Remove duplicate call of reset() function in CurrentOrigin.withOrigin().
## How was this patch tested?
Existing test cases.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16615 from jiangxb1987/dummy-code.
### What changes were proposed in this pull request?
Empty partition column values are not valid for partition specification. Before this PR, we accept users to do it; however, Hive metastore does not detect and disallow it too. Thus, users hit the following strange error.
```Scala
val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name")
df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable")
spark.sql("alter table partitionedTable drop partition(partCol1='')")
spark.table("partitionedTable").show()
```
In the above example, the WHOLE table is DROPPED when users specify a partition spec containing only one partition column with empty values.
When the partition columns contains more than one, Hive metastore APIs simply ignore the columns with empty values and treat it as partial spec. This is also not expected. This does not follow the actual Hive behaviors. This PR is to disallow users to specify such an invalid partition spec in the `SessionCatalog` APIs.
### How was this patch tested?
Added test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16583 from gatorsmile/disallowEmptyPartColValue.
## What changes were proposed in this pull request?
This PR is a follow-up to address the comments https://github.com/apache/spark/pull/16233/files#r95669988 and https://github.com/apache/spark/pull/16233/files#r95662299.
We try to wrap the child by:
1. Generate the `queryOutput` by:
1.1. If the query column names are defined, map the column names to attributes in the child output by name;
1.2. Else set the child output attributes to `queryOutput`.
2. Map the `queryQutput` to view output by index, if the corresponding attributes don't match, try to up cast and alias the attribute in `queryOutput` to the attribute in the view output.
3. Add a Project over the child, with the new output generated by the previous steps.
If the view output doesn't have the same number of columns neither with the child output, nor with the query column names, throw an AnalysisException.
## How was this patch tested?
Add new test cases in `SQLViewSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16561 from jiangxb1987/alias-view.
## What changes were proposed in this pull request?
When we convert a string to integral, we will convert that string to `decimal(20, 0)` first, so that we can turn a string with decimal format to truncated integral, e.g. `CAST('1.2' AS int)` will return `1`.
However, this brings problems when we convert a string with large numbers to integral, e.g. `CAST('1234567890123' AS int)` will return `1912276171`, while Hive returns null as we expected.
This is a long standing bug(seems it was there the first day Spark SQL was created), this PR fixes this bug by adding the native support to convert `UTF8String` to integral.
## How was this patch tested?
new regression tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16550 from cloud-fan/string-to-int.
## What changes were proposed in this pull request?
Pivoting adds backticks (e.g. 3_count(\`c\`)) in column names and, in some cases,
thes causes analysis exceptions like;
```
scala> val df = Seq((2, 3, 4), (3, 4, 5)).toDF("a", "x", "y")
scala> df.groupBy("a").pivot("x").agg(count("y"), avg("y")).na.fill(0)
org.apache.spark.sql.AnalysisException: syntax error in attribute name: `3_count(`y`)`;
at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.e$1(unresolved.scala:134)
at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.parseAttributeName(unresolved.scala:144)
...
```
So, this pr proposes to remove these backticks from column names.
## How was this patch tested?
Added a test in `DataFrameAggregateSuite`.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#14812 from maropu/SPARK-17237.
## What changes were proposed in this pull request?
In this pr, we add more test cases for project and aggregate estimation.
## How was this patch tested?
Add test cases.
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#16551 from wzhfy/addTests.
## What changes were proposed in this pull request?
This patch simplifies slightly the logical plan statistics cache implementation, as discussed in https://github.com/apache/spark/pull/16529
## How was this patch tested?
N/A - this has no behavior change.
Author: Reynold Xin <rxin@databricks.com>
Closes#16544 from rxin/SPARK-19149.
## What changes were proposed in this pull request?
We should be able to resolve a nested view. The main advantage is that if you update an underlying view, the current view also gets updated.
The new approach should be compatible with older versions of SPARK/HIVE, that means:
1. The new approach should be able to resolve the views that created by older versions of SPARK/HIVE;
2. The new approach should be able to resolve the views that are currently supported by SPARK SQL.
The new approach mainly brings in the following changes:
1. Add a new operator called `View` to keep track of the CatalogTable that describes the view, and the output attributes as well as the child of the view;
2. Update the `ResolveRelations` rule to resolve the relations and views, note that a nested view should be resolved correctly;
3. Add `viewDefaultDatabase` variable to `CatalogTable` to keep track of the default database name used to resolve a view, if the `CatalogTable` is not a view, then the variable should be `None`;
4. Add `AnalysisContext` to enable us to still support a view created with CTE/Windows query;
5. Enables the view support without enabling Hive support (i.e., enableHiveSupport);
6. Fix a weird behavior: the result of a view query may have different schema if the referenced table has been changed. After this PR, we try to cast the child output attributes to that from the view schema, throw an AnalysisException if cast is not allowed.
Note this is compatible with the views defined by older versions of Spark(before 2.2), which have empty `defaultDatabase` and all the relations in `viewText` have database part defined.
## How was this patch tested?
1. Add new tests in `SessionCatalogSuite` to test the function `lookupRelation`;
2. Add new test case in `SQLViewSuite` to test resolve a nested view.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16233 from jiangxb1987/resolve-view.
## What changes were proposed in this pull request?
Currently we have two sets of statistics in LogicalPlan: a simple stats and a stats estimated by cbo, but the computing logic and naming are quite confusing, we need to unify these two sets of stats.
## How was this patch tested?
Just modify existing tests.
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#16529 from wzhfy/unifyStats.
## What changes were proposed in this pull request?
This PR allow update mode for non-aggregation streaming queries. It will be same as the append mode if a query has no aggregations.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16520 from zsxwing/update-without-agg.
## What changes were proposed in this pull request?
Support cardinality estimation of aggregate operator
## How was this patch tested?
Add test cases
Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#16431 from wzhfy/aggEstimation.
## What changes were proposed in this pull request?
Support cardinality estimation for project operator.
## How was this patch tested?
Add a test suite and a base class in the catalyst package.
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#16430 from wzhfy/projectEstimation.
## What changes were proposed in this pull request?
Added a `to` call at the end of the code generated by `ScalaReflection.deserializerFor` if the requested type is not a supertype of `WrappedArray[_]` that uses `CanBuildFrom[_, _, _]` to convert result into an arbitrary subtype of `Seq[_]`.
Care was taken to preserve the original deserialization where it is possible to avoid the overhead of conversion in cases where it is not needed
`ScalaReflection.serializerFor` could already be used to serialize any `Seq[_]` so it was not altered
`SQLImplicits` had to be altered and new implicit encoders added to permit serialization of other sequence types
Also fixes [SPARK-16815] Dataset[List[T]] leads to ArrayStoreException
## How was this patch tested?
```bash
./build/mvn -DskipTests clean package && ./dev/run-tests
```
Also manual execution of the following sets of commands in the Spark shell:
```scala
case class TestCC(key: Int, letters: List[String])
val ds1 = sc.makeRDD(Seq(
(List("D")),
(List("S","H")),
(List("F","H")),
(List("D","L","L"))
)).map(x=>(x.length,x)).toDF("key","letters").as[TestCC]
val test1=ds1.map{_.key}
test1.show
```
```scala
case class X(l: List[String])
spark.createDataset(Seq(List("A"))).map(X).show
```
```scala
spark.sqlContext.createDataset(sc.parallelize(List(1) :: Nil)).collect
```
After adding arbitrary sequence support also tested with the following commands:
```scala
case class QueueClass(q: scala.collection.immutable.Queue[Int])
spark.createDataset(Seq(List(1,2,3))).map(x => QueueClass(scala.collection.immutable.Queue(x: _*))).map(_.q.dequeue).collect
```
Author: Michal Senkyr <mike.senkyr@gmail.com>
Closes#16240 from michalsenkyr/sql-caseclass-list-fix.
## What changes were proposed in this pull request?
Today we have different syntax to create data source or hive serde tables, we should unify them to not confuse users and step forward to make hive a data source.
Please read https://issues.apache.org/jira/secure/attachment/12843835/CREATE-TABLE.pdf for details.
TODO(for follow-up PRs):
1. TBLPROPERTIES is not added to the new syntax, we should decide if we wanna add it later.
2. `SHOW CREATE TABLE` should be updated to use the new syntax.
3. we should decide if we wanna change the behavior of `SET LOCATION`.
## How was this patch tested?
new tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16296 from cloud-fan/create-table.
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.
## How was this patch tested?
N/A since only docs or comments were updated.
Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com>
Closes#16455 from neurons/np.structure_streaming_doc.
## What changes were proposed in this pull request?
Now all aggregation functions support partial aggregate, we can remove the `supportsPartual` flag in `AggregateFunction`
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16461 from cloud-fan/partial.
## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/16402 we made a mistake that, when double/float is infinity, the `Literal` codegen will output boxed value and cause wrong result.
This PR fixes this by special handling infinity to not output boxed value.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16469 from cloud-fan/literal.
### What changes were proposed in this pull request?
The data in the managed table should be deleted after table is dropped. However, if the partition location is not under the location of the partitioned table, it is not deleted as expected. Users can specify any location for the partition when they adding a partition.
This PR is to delete partition location when dropping managed partitioned tables stored in `InMemoryCatalog`.
### How was this patch tested?
Added test cases for both HiveExternalCatalog and InMemoryCatalog
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16448 from gatorsmile/unsetSerdeProp.
## What changes were proposed in this pull request?
Currently collect_set/collect_list aggregation expression don't support partial aggregation. This patch is to enable partial aggregation for them.
## How was this patch tested?
Jenkins tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#16371 from viirya/collect-partial-support.
## What changes were proposed in this pull request?
We add a cbo configuration to switch between default stats and estimated stats.
We also define a new statistics method `planStats` in LogicalPlan with conf as its parameter, in order to pass the cbo switch and other estimation related configurations in the future. `planStats` is used on the caller sides (i.e. in Optimizer and Strategies) to make transformation decisions based on stats.
## How was this patch tested?
Add a test case using a dummy LogicalPlan.
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#16401 from wzhfy/cboSwitch.
### What changes were proposed in this pull request?
Remove useless `databaseName ` from `SimpleCatalogRelation`.
### How was this patch tested?
Existing test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16438 from gatorsmile/removeDBFromSimpleCatalogRelation.
### What changes were proposed in this pull request?
Fixed non-thread-safe functions used in SessionCatalog:
- refreshTable
- lookupRelation
### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16437 from gatorsmile/addSyncToLookUpTable.
## What changes were proposed in this pull request?
This PR proposes to fix the test failures due to different format of paths on Windows.
Failed tests are as below:
```
ColumnExpressionSuite:
- input_file_name, input_file_block_start, input_file_block_length - FileScanRDD *** FAILED *** (187 milliseconds)
"file:///C:/projects/spark/target/tmp/spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce/part-00001-c083a03a-e55e-4b05-9073-451de352d006.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce" (ColumnExpressionSuite.scala:545)
- input_file_name, input_file_block_start, input_file_block_length - HadoopRDD *** FAILED *** (172 milliseconds)
"file:/C:/projects/spark/target/tmp/spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f/part-00000-f6530138-9ad3-466d-ab46-0eeb6f85ed0b.txt" did not contain "C:\projects\spark\target\tmp\spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f" (ColumnExpressionSuite.scala:569)
- input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD *** FAILED *** (156 milliseconds)
"file:/C:/projects/spark/target/tmp/spark-a894c7df-c74d-4d19-82a2-a04744cb3766/part-00000-29674e3f-3fcf-4327-9b04-4dab1d46338d.txt" did not contain "C:\projects\spark\target\tmp\spark-a894c7df-c74d-4d19-82a2-a04744cb3766" (ColumnExpressionSuite.scala:598)
```
```
DataStreamReaderWriterSuite:
- source metadataPath *** FAILED *** (62 milliseconds)
org.mockito.exceptions.verification.junit.ArgumentsAreDifferent: Argument(s) are different! Wanted:
streamSourceProvider.createSource(
org.apache.spark.sql.SQLContext3b04133b,
"C:\projects\spark\target\tmp\streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0",
None,
"org.apache.spark.sql.streaming.test",
Map()
);
-> at org.apache.spark.sql.streaming.test.DataStreamReaderWriterSuite$$anonfun$12.apply$mcV$sp(DataStreamReaderWriterSuite.scala:374)
Actual invocation has different arguments:
streamSourceProvider.createSource(
org.apache.spark.sql.SQLContext3b04133b,
"/C:/projects/spark/target/tmp/streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0",
None,
"org.apache.spark.sql.streaming.test",
Map()
);
```
```
GlobalTempViewSuite:
- CREATE GLOBAL TEMP VIEW USING *** FAILED *** (110 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-960398ba-a0a1-45f6-a59a-d98533f9f519;
```
```
CreateTableAsSelectSuite:
- CREATE TABLE USING AS SELECT *** FAILED *** (0 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
- create a table, drop it and create another one with the same name *** FAILED *** (16 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
- create table using as select - with partitioned by *** FAILED *** (0 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
- create table using as select - with non-zero buckets *** FAILED *** (0 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
```
```
HiveMetadataCacheSuite:
- partitioned table is cached when partition pruning is true *** FAILED *** (532 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- partitioned table is cached when partition pruning is false *** FAILED *** (297 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```
```
MultiDatabaseSuite:
- createExternalTable() to non-default database - with USE *** FAILED *** (954 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-0839d9a7-5e29-467a-9e3e-3e4cd618ee09;
- createExternalTable() to non-default database - without USE *** FAILED *** (500 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c7e24d73-1d8f-45e8-ab7d-53a83087aec3;
- invalid database name and table names *** FAILED *** (31 milliseconds)
"Path does not exist: file:/C:projectsspark arget mpspark-15a2a494-3483-4876-80e5-ec396e704b77;" did not contain "`t:a` is not a valid name for tables/databases. Valid names only contain alphabet characters, numbers and _." (MultiDatabaseSuite.scala:296)
```
```
OrcQuerySuite:
- SPARK-8501: Avoids discovery schema from empty ORC files *** FAILED *** (15 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- Verify the ORC conversion parameter: CONVERT_METASTORE_ORC *** FAILED *** (78 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- converted ORC table supports resolving mixed case field *** FAILED *** (297 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```
```
HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite:
- Locality support for FileScanRDD *** FAILED *** (15 milliseconds)
java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-383d1f13-8783-47fd-964d-9c75e5eec50f, expected: file:///
```
```
HiveQuerySuite:
- CREATE TEMPORARY FUNCTION *** FAILED *** (0 milliseconds)
java.net.MalformedURLException: For input string: "%5Cprojects%5Cspark%5Csql%5Chive%5Ctarget%5Cscala-2.11%5Ctest-classes%5CTestUDTF.jar"
- ADD FILE command *** FAILED *** (500 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\sql\hive\target\scala-2.11\test-classes\data\files\v1.txt
- ADD JAR command 2 *** FAILED *** (110 milliseconds)
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilessample.json;
```
```
PruneFileSourcePartitionsSuite:
- PruneFileSourcePartitions should not change the output of LogicalRelation *** FAILED *** (15 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```
```
HiveCommandSuite:
- LOAD DATA LOCAL *** FAILED *** (109 milliseconds)
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilesemployee.dat;
- LOAD DATA *** FAILED *** (93 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark arget mpemployee.dat7496657117354281006.tmp
- Truncate Table *** FAILED *** (78 milliseconds)
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilesemployee.dat;
```
```
HiveExternalCatalogBackwardCompatibilitySuite:
- make sure we can read table created by old version of Spark *** FAILED *** (0 milliseconds)
"[/C:/projects/spark/target/tmp/]spark-0554d859-74e1-..." did not equal "[C:\projects\spark\target\tmp\]spark-0554d859-74e1-..." (HiveExternalCatalogBackwardCompatibilitySuite.scala:213)
org.scalatest.exceptions.TestFailedException
- make sure we can alter table location created by old version of Spark *** FAILED *** (110 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark arget mpspark-0e9b2c5f-49a1-4e38-a32a-c0ab1813a79f
```
```
ExternalCatalogSuite:
- create/drop/rename partitions should create/delete/rename the directory *** FAILED *** (610 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-4c24f010-18df-437b-9fed-990c6f9adece
```
```
SQLQuerySuite:
- describe functions - temporary user defined functions *** FAILED *** (16 milliseconds)
java.net.URISyntaxException: Illegal character in opaque part at index 22: C:projectssparksqlhive argetscala-2.11 est-classesTestUDTF.jar
- specifying database name for a temporary table is not allowed *** FAILED *** (125 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-a34c9814-a483-43f2-be29-37f616b6df91;
```
```
PartitionProviderCompatibilitySuite:
- convert partition provider to hive with repair table *** FAILED *** (281 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-ee5fc96d-8c7d-4ebf-8571-a1d62736473e;
- when partition management is enabled, new tables have partition provider hive *** FAILED *** (187 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-803ad4d6-3e8c-498d-9ca5-5cda5d9b2a48;
- when partition management is disabled, new tables have no partition provider *** FAILED *** (172 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c9fda9e2-4020-465f-8678-52cd72d0a58f;
- when partition management is disabled, we preserve the old behavior even for new tables *** FAILED *** (203 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget
mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e13;
- insert overwrite partition of legacy datasource table *** FAILED *** (188 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e79;
- insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (219 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-6ba3a88d-6f6c-42c5-a9f4-6d924a0616ff;
- SPARK-18544 append with saveAsTable - partition management true *** FAILED *** (173 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-cd234a6d-9cb4-4d1d-9e51-854ae9543bbd;
- SPARK-18635 special chars in partition values - partition management true *** FAILED *** (2 seconds, 967 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- SPARK-18635 special chars in partition values - partition management false *** FAILED *** (62 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- SPARK-18659 insert overwrite table with lowercase - partition management true *** FAILED *** (63 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- SPARK-18544 append with saveAsTable - partition management false *** FAILED *** (266 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- SPARK-18659 insert overwrite table files - partition management false *** FAILED *** (63 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- SPARK-18659 insert overwrite table with lowercase - partition management false *** FAILED *** (78 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- sanity check table setup *** FAILED *** (31 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- insert into partial dynamic partitions *** FAILED *** (47 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- insert into fully dynamic partitions *** FAILED *** (62 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- insert into static partition *** FAILED *** (78 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- overwrite partial dynamic partitions *** FAILED *** (63 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- overwrite fully dynamic partitions *** FAILED *** (47 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- overwrite static partition *** FAILED *** (63 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```
```
MetastoreDataSourcesSuite:
- check change without refresh *** FAILED *** (203 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-00713fe4-ca04-448c-bfc7-6c5e9a2ad2a1;
- drop, change, recreate *** FAILED *** (78 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-2030a21b-7d67-4385-a65b-bb5e2bed4861;
- SPARK-15269 external data source table creation *** FAILED *** (78 milliseconds)
org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4d50fd4a-14bc-41d6-9232-9554dd233f86;
- CTAS *** FAILED *** (109 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
- CTAS with IF NOT EXISTS *** FAILED *** (109 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
- CTAS: persisted partitioned bucketed data source table *** FAILED *** (0 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
- SPARK-15025: create datasource table with path with select *** FAILED *** (16 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
- CTAS: persisted partitioned data source table *** FAILED *** (47 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
```
```
HiveMetastoreCatalogSuite:
- Persist non-partitioned parquet relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
- Persist non-partitioned orc relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds)
java.lang.IllegalArgumentException: Can not create a Path from an empty string
```
```
HiveUDFSuite:
- SPARK-11522 select input_file_name from non-parquet table *** FAILED *** (16 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```
```
QueryPartitionSuite:
- SPARK-13709: reading partitioned Avro table with nested schema *** FAILED *** (250 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```
```
ParquetHiveCompatibilitySuite:
- simple primitives *** FAILED *** (16 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- SPARK-10177 timestamp *** FAILED *** (0 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- array *** FAILED *** (16 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- map *** FAILED *** (16 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- struct *** FAILED *** (0 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
- SPARK-16344: array of struct with a single field named 'array_element' *** FAILED *** (15 milliseconds)
org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```
## How was this patch tested?
Manually tested via AppVeyor.
```
ColumnExpressionSuite:
- input_file_name, input_file_block_start, input_file_block_length - FileScanRDD (234 milliseconds)
- input_file_name, input_file_block_start, input_file_block_length - HadoopRDD (235 milliseconds)
- input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD (203 milliseconds)
```
```
DataStreamReaderWriterSuite:
- source metadataPath (63 milliseconds)
```
```
GlobalTempViewSuite:
- CREATE GLOBAL TEMP VIEW USING (436 milliseconds)
```
```
CreateTableAsSelectSuite:
- CREATE TABLE USING AS SELECT (171 milliseconds)
- create a table, drop it and create another one with the same name (422 milliseconds)
- create table using as select - with partitioned by (141 milliseconds)
- create table using as select - with non-zero buckets (125 milliseconds)
```
```
HiveMetadataCacheSuite:
- partitioned table is cached when partition pruning is true (3 seconds, 211 milliseconds)
- partitioned table is cached when partition pruning is false (1 second, 781 milliseconds)
```
```
MultiDatabaseSuite:
- createExternalTable() to non-default database - with USE (797 milliseconds)
- createExternalTable() to non-default database - without USE (640 milliseconds)
- invalid database name and table names (62 milliseconds)
```
```
OrcQuerySuite:
- SPARK-8501: Avoids discovery schema from empty ORC files (703 milliseconds)
- Verify the ORC conversion parameter: CONVERT_METASTORE_ORC (750 milliseconds)
- converted ORC table supports resolving mixed case field (625 milliseconds)
```
```
HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite:
- Locality support for FileScanRDD (296 milliseconds)
```
```
HiveQuerySuite:
- CREATE TEMPORARY FUNCTION (125 milliseconds)
- ADD FILE command (250 milliseconds)
- ADD JAR command 2 (609 milliseconds)
```
```
PruneFileSourcePartitionsSuite:
- PruneFileSourcePartitions should not change the output of LogicalRelation (359 milliseconds)
```
```
HiveCommandSuite:
- LOAD DATA LOCAL (1 second, 829 milliseconds)
- LOAD DATA (1 second, 735 milliseconds)
- Truncate Table (1 second, 641 milliseconds)
```
```
HiveExternalCatalogBackwardCompatibilitySuite:
- make sure we can read table created by old version of Spark (32 milliseconds)
- make sure we can alter table location created by old version of Spark (125 milliseconds)
- make sure we can rename table created by old version of Spark (281 milliseconds)
```
```
ExternalCatalogSuite:
- create/drop/rename partitions should create/delete/rename the directory (625 milliseconds)
```
```
SQLQuerySuite:
- describe functions - temporary user defined functions (31 milliseconds)
- specifying database name for a temporary table is not allowed (390 milliseconds)
```
```
PartitionProviderCompatibilitySuite:
- convert partition provider to hive with repair table (813 milliseconds)
- when partition management is enabled, new tables have partition provider hive (562 milliseconds)
- when partition management is disabled, new tables have no partition provider (344 milliseconds)
- when partition management is disabled, we preserve the old behavior even for new tables (422 milliseconds)
- insert overwrite partition of legacy datasource table (750 milliseconds)
- SPARK-18544 append with saveAsTable - partition management true (985 milliseconds)
- SPARK-18635 special chars in partition values - partition management true (3 seconds, 328 milliseconds)
- SPARK-18635 special chars in partition values - partition management false (2 seconds, 891 milliseconds)
- SPARK-18659 insert overwrite table with lowercase - partition management true (750 milliseconds)
- SPARK-18544 append with saveAsTable - partition management false (656 milliseconds)
- SPARK-18659 insert overwrite table files - partition management false (922 milliseconds)
- SPARK-18659 insert overwrite table with lowercase - partition management false (469 milliseconds)
- sanity check table setup (937 milliseconds)
- insert into partial dynamic partitions (2 seconds, 985 milliseconds)
- insert into fully dynamic partitions (1 second, 937 milliseconds)
- insert into static partition (1 second, 578 milliseconds)
- overwrite partial dynamic partitions (7 seconds, 561 milliseconds)
- overwrite fully dynamic partitions (1 second, 766 milliseconds)
- overwrite static partition (1 second, 797 milliseconds)
```
```
MetastoreDataSourcesSuite:
- check change without refresh (610 milliseconds)
- drop, change, recreate (437 milliseconds)
- SPARK-15269 external data source table creation (297 milliseconds)
- CTAS with IF NOT EXISTS (437 milliseconds)
- CTAS: persisted partitioned bucketed data source table (422 milliseconds)
- SPARK-15025: create datasource table with path with select (265 milliseconds)
- CTAS (438 milliseconds)
- CTAS with IF NOT EXISTS (469 milliseconds)
- CTAS: persisted partitioned bucketed data source table (406 milliseconds)
```
```
HiveMetastoreCatalogSuite:
- Persist non-partitioned parquet relation into metastore as managed table using CTAS (406 milliseconds)
- Persist non-partitioned orc relation into metastore as managed table using CTAS (313 milliseconds)
```
```
HiveUDFSuite:
- SPARK-11522 select input_file_name from non-parquet table (3 seconds, 144 milliseconds)
```
```
QueryPartitionSuite:
- SPARK-13709: reading partitioned Avro table with nested schema (1 second, 67 milliseconds)
```
```
ParquetHiveCompatibilitySuite:
- simple primitives (745 milliseconds)
- SPARK-10177 timestamp (375 milliseconds)
- array (407 milliseconds)
- map (409 milliseconds)
- struct (437 milliseconds)
- SPARK-16344: array of struct with a single field named 'array_element' (391 milliseconds)
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16397 from HyukjinKwon/SPARK-18922-paths.
## What changes were proposed in this pull request?
`Literal` can use `CodegenContex.addReferenceObj` to implement codegen, instead of `CodegenFallback`. This can also simplify the generated code a little bit, before we will generate: `((Expression) references[1]).eval(null)`, now it's just `references[1]`.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16402 from cloud-fan/minor.
## What changes were proposed in this pull request?
Currently we implement `Aggregator` with `DeclarativeAggregate`, which will serialize/deserialize the buffer object every time we process an input.
This PR implements `Aggregator` with `TypedImperativeAggregate` and avoids to serialize/deserialize buffer object many times. The benchmark shows we get about 2 times speed up.
For simple buffer object that doesn't need serialization, we still go with `DeclarativeAggregate`, to avoid performance regression.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16383 from cloud-fan/aggregator.
## What changes were proposed in this pull request?
Statistics in LogicalPlan should use attributes to refer to columns rather than column names, because two columns from two relations can have the same column name. But CatalogTable doesn't have the concepts of attribute or broadcast hint in Statistics. Therefore, putting Statistics in CatalogTable is confusing.
We define a different statistic structure in CatalogTable, which is only responsible for interacting with metastore, and is converted to statistics in LogicalPlan when it is used.
## How was this patch tested?
add test cases
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#16323 from wzhfy/nameToAttr.
## What changes were proposed in this pull request?
SortPartitions and RedistributeData logical operators are not actually used and can be removed. Note that we do have a Sort operator (with global flag false) that subsumed SortPartitions.
## How was this patch tested?
Also updated test cases to reflect the removal.
Author: Reynold Xin <rxin@databricks.com>
Closes#16381 from rxin/SPARK-18973.
## What changes were proposed in this pull request?
Made update mode public. As part of that here are the changes.
- Update DatastreamWriter to accept "update"
- Changed package of InternalOutputModes from o.a.s.sql to o.a.s.sql.catalyst
- Added update mode state removing with watermark to StateStoreSaveExec
## How was this patch tested?
Added new tests in changed modules
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#16360 from tdas/SPARK-18234.
## What changes were proposed in this pull request?
When we append data to an existing table with `DataFrameWriter.saveAsTable`, we will do various checks to make sure the appended data is consistent with the existing data.
However, we get the information of the existing table by matching the table relation, instead of looking at the table metadata. This is error-prone, e.g. we only check the number of columns for `HadoopFsRelation`, we forget to check bucketing, etc.
This PR refactors the error checking by looking at the metadata of the existing table, and fix several bugs:
* SPARK-18899: We forget to check if the specified bucketing matched the existing table, which may lead to a problematic table that has different bucketing in different data files.
* SPARK-18912: We forget to check the number of columns for non-file-based data source table
* SPARK-18913: We don't support append data to a table with special column names.
## How was this patch tested?
new regression test.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16313 from cloud-fan/bug1.
## What changes were proposed in this pull request?
Currently `ImplicitTypeCasts` doesn't handle casts between `ArrayType`s, this is not convenient, we should add a rule to enable casting from `ArrayType(InternalType)` to `ArrayType(newInternalType)`.
Goals:
1. Add a rule to `ImplicitTypeCasts` to enable casting between `ArrayType`s;
2. Simplify `Percentile` and `ApproximatePercentile`.
## How was this patch tested?
Updated test cases in `TypeCoercionSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16057 from jiangxb1987/implicit-cast-complex-types.
## What changes were proposed in this pull request?
percentile_approx is the name used in Hive, and approx_percentile is the name used in Presto. approx_percentile is actually more consistent with our approx_count_distinct. Given the cost to alias SQL functions is low (one-liner), it'd be better to just alias them so it is easier to use.
## How was this patch tested?
Technically I could add an end-to-end test to verify this one-line change, but it seemed too trivial to me.
Author: Reynold Xin <rxin@databricks.com>
Closes#16300 from rxin/SPARK-18892.
## What changes were proposed in this pull request?
Check whether Aggregation operators on a streaming subplan have aggregate expressions with isDistinct = true.
## How was this patch tested?
Added unit test
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#16289 from tdas/SPARK-18870.
## What changes were proposed in this pull request?
Right now, once a user set the comment of a column with create table command, he/she cannot update the comment. It will be useful to provide a public interface (e.g. SQL) to do that.
This PR implements the following SQL statement:
```
ALTER TABLE table [PARTITION partition_spec]
CHANGE [COLUMN] column_old_name column_new_name column_dataType
[COMMENT column_comment]
[FIRST | AFTER column_name];
```
For further expansion, we could support alter `name`/`dataType`/`index` of a column too.
## How was this patch tested?
Add new test cases in `ExternalCatalogSuite` and `SessionCatalogSuite`.
Add sql file test for `ALTER TABLE CHANGE COLUMN` statement.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15717 from jiangxb1987/change-column.
## What changes were proposed in this pull request?
After the bug fix in SPARK-18854, TreeNode.apply now returns TreeNode[_] rather than a more specific type. It would be easier for interactive debugging to introduce a function that returns the BaseType.
## How was this patch tested?
N/A - this is a developer only feature used for interactive debugging. As long as it compiles, it should be good to go. I tested this in spark-shell.
Author: Reynold Xin <rxin@databricks.com>
Closes#16288 from rxin/SPARK-18869.
## What changes were proposed in this pull request?
This is a bug introduced by subquery handling. numberedTreeString (which uses generateTreeString under the hood) numbers trees including innerChildren (used to print subqueries), but apply (which uses getNodeNumbered) ignores innerChildren. As a result, apply(i) would return the wrong plan node if there are subqueries.
This patch fixes the bug.
## How was this patch tested?
Added a test case in SubquerySuite.scala to test both the depth-first traversal of numbering as well as making sure the two methods are consistent.
Author: Reynold Xin <rxin@databricks.com>
Closes#16277 from rxin/SPARK-18854.
## What changes were proposed in this pull request?
This patch reduces the default number element estimation for arrays and maps from 100 to 1. The issue with the 100 number is that when nested (e.g. an array of map), 100 * 100 would be used as the default size. This sounds like just an overestimation which doesn't seem that bad (since it is usually better to overestimate than underestimate). However, due to the way we assume the size output for Project (new estimated column size / old estimated column size), this overestimation can become underestimation. It is actually in general in this case safer to assume 1 default element.
## How was this patch tested?
This should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#16274 from rxin/SPARK-18853.
## What changes were proposed in this pull request?
Move the checking of GROUP BY column in correlated scalar subquery from CheckAnalysis
to Analysis to fix a regression caused by SPARK-18504.
This problem can be reproduced with a simple script now.
Seq((1,1)).toDF("pk","pv").createOrReplaceTempView("p")
Seq((1,1)).toDF("ck","cv").createOrReplaceTempView("c")
sql("select * from p,c where p.pk=c.ck and c.cv = (select avg(c1.cv) from c c1 where c1.ck = p.pk)").show
The requirements are:
1. We need to reference the same table twice in both the parent and the subquery. Here is the table c.
2. We need to have a correlated predicate but to a different table. Here is from c (as c1) in the subquery to p in the parent.
3. We will then "deduplicate" c1.ck in the subquery to `ck#<n1>#<n2>` at `Project` above `Aggregate` of `avg`. Then when we compare `ck#<n1>#<n2>` and the original group by column `ck#<n1>` by their canonicalized form, which is #<n2> != #<n1>. That's how we trigger the exception added in SPARK-18504.
## How was this patch tested?
SubquerySuite and a simplified version of TPCDS-Q32
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#16246 from nsyca/18814.
## What changes were proposed in this pull request?
`OverwriteOptions` was introduced in https://github.com/apache/spark/pull/15705, to carry the information of static partitions. However, after further refactor, this information becomes duplicated and we can remove `OverwriteOptions`.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15995 from cloud-fan/overwrite.
## What changes were proposed in this pull request?
Change the statement `SHOW TABLES [EXTENDED] [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards'] [PARTITION(partition_spec)]` to the following statements:
- SHOW TABLES [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards']
- SHOW TABLE EXTENDED [(IN|FROM) database_name] LIKE 'identifier_with_wildcards' [PARTITION(partition_spec)]
After this change, the statements `SHOW TABLE/SHOW TABLES` have the same syntax with that HIVE has.
## How was this patch tested?
Modified the test sql file `show-tables.sql`;
Modified the test suite `DDLSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#16262 from jiangxb1987/show-table-extended.
## What changes were proposed in this pull request?
Fixes compile errors in generated code when user has case class with a `scala.collections.immutable.Map` instead of a `scala.collections.Map`. Since ArrayBasedMapData.toScalaMap returns the immutable version we can make it work with both.
## How was this patch tested?
Additional unit tests.
Author: Andrew Ray <ray.andrew@gmail.com>
Closes#16161 from aray/fix-map-codegen.
The value of the "isSrcLocal" parameter passed to Hive's loadTable and
loadPartition methods needs to be set according to the user query (e.g.
"LOAD DATA LOCAL"), and not the current code that tries to guess what
it should be.
For existing versions of Hive the current behavior is probably ok, but
some recent changes in the Hive code changed the semantics slightly,
making code that sets "isSrcLocal" to "true" incorrectly to do the
wrong thing. It would end up moving the parent directory of the files
into the final location, instead of the file themselves, resulting
in a table that cannot be read.
I modified HiveCommandSuite so that existing "LOAD DATA" tests are run
both in local and non-local mode, since the semantics are slightly different.
The tests include a few new checks to make sure the semantics follow
what Hive describes in its documentation.
Tested with existing unit tests and also ran some Hive integration tests
with a version of Hive containing the changes that surfaced the problem.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#16179 from vanzin/SPARK-18752.
## What changes were proposed in this pull request?
After https://github.com/apache/spark/pull/15620 , all of the Maven-based 2.0 Jenkins jobs time out consistently. As I pointed out in https://github.com/apache/spark/pull/15620#discussion_r91829129 , it seems that the regression test is an overkill and may hit constants pool size limitation, which is a known issue and hasn't been fixed yet.
Since #15620 only fix the code size limitation problem, we can simplify the test to avoid hitting constants pool size limitation.
## How was this patch tested?
test only change
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16244 from cloud-fan/minor.
## What changes were proposed in this pull request?
During column stats collection, average and max length will be null if a column of string/binary type has only null values. To fix this, I use default size when avg/max length is null.
## How was this patch tested?
Add a test for handling null columns
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#16243 from wzhfy/nullStats.
## What changes were proposed in this pull request?
1. In SparkStrategies.canBroadcast, I will add the check plan.statistics.sizeInBytes >= 0
2. In LocalRelations.statistics, when calculate the statistics, I will change the size to BigInt so it won't overflow.
## How was this patch tested?
I will add a test case to make sure the statistics.sizeInBytes won't overflow.
Author: Huaxin Gao <huaxing@us.ibm.com>
Closes#16175 from huaxingao/spark-17460.
## What changes were proposed in this pull request?
Typo fixes
## How was this patch tested?
Local build. Awaiting the official build.
Author: Jacek Laskowski <jacek@japila.pl>
Closes#16144 from jaceklaskowski/typo-fixes.
## What changes were proposed in this pull request?
`makeRootConverter` is only called with a `StructType` value. By making this method less general we can remove pattern matches, which are never actually hit outside of the test suite.
## How was this patch tested?
The existing tests.
Author: Nathan Howell <nhowell@godaddy.com>
Closes#16084 from NathanHowell/SPARK-18654.
## What changes were proposed in this pull request?
Fixes AnalysisException for pivot queries that have group by columns that are expressions and not attributes by substituting the expressions output attribute in the second aggregation and final projection.
## How was this patch tested?
existing and additional unit tests
Author: Andrew Ray <ray.andrew@gmail.com>
Closes#16177 from aray/SPARK-17760.
## What changes were proposed in this pull request?
I jumped the gun on merging https://github.com/apache/spark/pull/16120, and missed a tiny potential problem. This PR fixes that by changing a val into a def; this should prevent potential serialization/initialization weirdness from happening.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16170 from hvanhovell/SPARK-18634.
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-18572)
## What changes were proposed in this pull request?
Currently Spark answers the `SHOW PARTITIONS` command by fetching all of the table's partition metadata from the external catalog and constructing partition names therefrom. The Hive client has a `getPartitionNames` method which is many times faster for this purpose, with the performance improvement scaling with the number of partitions in a table.
To test the performance impact of this PR, I ran the `SHOW PARTITIONS` command on two Hive tables with large numbers of partitions. One table has ~17,800 partitions, and the other has ~95,000 partitions. For the purposes of this PR, I'll call the former table `table1` and the latter table `table2`. I ran 5 trials for each table with before-and-after versions of this PR. The results are as follows:
Spark at bdc8153, `SHOW PARTITIONS table1`, times in seconds:
7.901
3.983
4.018
4.331
4.261
Spark at bdc8153, `SHOW PARTITIONS table2`
(Timed out after 10 minutes with a `SocketTimeoutException`.)
Spark at this PR, `SHOW PARTITIONS table1`, times in seconds:
3.801
0.449
0.395
0.348
0.336
Spark at this PR, `SHOW PARTITIONS table2`, times in seconds:
5.184
1.63
1.474
1.519
1.41
Taking the best times from each trial, we get a 12x performance improvement for a table with ~17,800 partitions and at least a 426x improvement for a table with ~95,000 partitions. More significantly, the latter command doesn't even complete with the current code in master.
This is actually a patch we've been using in-house at VideoAmp since Spark 1.1. It's made all the difference in the practical usability of our largest tables. Even with tables with about 1,000 partitions there's a performance improvement of about 2-3x.
## How was this patch tested?
I added a unit test to `VersionsSuite` which tests that the Hive client's `getPartitionNames` method returns the correct number of partitions.
Author: Michael Allman <michael@videoamp.com>
Closes#15998 from mallman/spark-18572-list_partition_names.
## What changes were proposed in this pull request?
As reported in the Jira, there are some weird issues with exploding Python UDFs in SparkSQL.
The following test code can reproduce it. Notice: the following test code is reported to return wrong results in the Jira. However, as I tested on master branch, it causes exception and so can't return any result.
>>> from pyspark.sql.functions import *
>>> from pyspark.sql.types import *
>>>
>>> df = spark.range(10)
>>>
>>> def return_range(value):
... return [(i, str(i)) for i in range(value - 1, value + 1)]
...
>>> range_udf = udf(return_range, ArrayType(StructType([StructField("integer_val", IntegerType()),
... StructField("string_val", StringType())])))
>>>
>>> df.select("id", explode(range_udf(df.id))).show()
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/spark/python/pyspark/sql/dataframe.py", line 318, in show
print(self._jdf.showString(n, 20))
File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 1133, in __call__
File "/spark/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/protocol.py", line 319, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o126.showString.: java.lang.AssertionError: assertion failed
at scala.Predef$.assert(Predef.scala:156)
at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:120)
at org.apache.spark.sql.execution.GenerateExec.consume(GenerateExec.scala:57)
The cause of this issue is, in `ExtractPythonUDFs` we insert `BatchEvalPythonExec` to run PythonUDFs in batch. `BatchEvalPythonExec` will add extra outputs (e.g., `pythonUDF0`) to original plan. In above case, the original `Range` only has one output `id`. After `ExtractPythonUDFs`, the added `BatchEvalPythonExec` has two outputs `id` and `pythonUDF0`.
Because the output of `GenerateExec` is given after analysis phase, in above case, it is the combination of `id`, i.e., the output of `Range`, and `col`. But in planning phase, we change `GenerateExec`'s child plan to `BatchEvalPythonExec` with additional output attributes.
It will cause no problem in non wholestage codegen. Because when evaluating the additional attributes are projected out the final output of `GenerateExec`.
However, as `GenerateExec` now supports wholestage codegen, the framework will input all the outputs of the child plan to `GenerateExec`. Then when consuming `GenerateExec`'s output data (i.e., calling `consume`), the number of output attributes is different to the output variables in wholestage codegen.
To solve this issue, this patch only gives the generator's output to `GenerateExec` after analysis phase. `GenerateExec`'s output is the combination of its child plan's output and the generator's output. So when we change `GenerateExec`'s child, its output is still correct.
## How was this patch tested?
Added test cases to PySpark.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#16120 from viirya/fix-py-udf-with-generator.
## What changes were proposed in this pull request?
This is kind of a long-standing bug, it's hidden until https://github.com/apache/spark/pull/15780 , which may add `AssertNotNull` on top of `LambdaVariable` and thus enables subexpression elimination.
However, subexpression elimination will evaluate the common expressions at the beginning, which is invalid for `LambdaVariable`. `LambdaVariable` usually represents loop variable, which can't be evaluated ahead of the loop.
This PR skips expressions containing `LambdaVariable` when doing subexpression elimination.
## How was this patch tested?
updated test in `DatasetAggregatorSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16143 from cloud-fan/aggregator.
## What changes were proposed in this pull request?
We currently have function input_file_name to get the path of the input file, but don't have functions to get the block start offset and length. This patch introduces two functions:
1. input_file_block_start: returns the file block start offset, or -1 if not available.
2. input_file_block_length: returns the file block length, or -1 if not available.
## How was this patch tested?
Updated existing test cases in ColumnExpressionSuite that covered input_file_name to also cover the two new functions.
Author: Reynold Xin <rxin@databricks.com>
Closes#16133 from rxin/SPARK-18702.
## What changes were proposed in this pull request?
Fix for SPARK-18091 which is a bug related to large if expressions causing generated SpecificUnsafeProjection code to exceed JVM code size limit.
This PR changes if expression's code generation to place its predicate, true value and false value expressions' generated code in separate methods in context so as to never generate too long combined code.
## How was this patch tested?
Added a unit test and also tested manually with the application (having transformations similar to the unit test) which caused the issue to be identified in the first place.
Author: Kapil Singh <kapsingh@adobe.com>
Closes#15620 from kapilsingh5050/SPARK-18091-IfCodegenFix.
## What changes were proposed in this pull request?
This fix puts an explicit list of operators that Spark supports for correlated subqueries.
## How was this patch tested?
Run sql/test, catalyst/test and add a new test case on Generate.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#16046 from nsyca/spark18455.0.
## What changes were proposed in this pull request?
This fixes the parser rule to match named expressions, which doesn't work for two reasons:
1. The name match is not coerced to a regular expression (missing .r)
2. The surrounding literals are incorrect and attempt to escape a single quote, which is unnecessary
## How was this patch tested?
This adds test cases for named expressions using the bracket syntax, including one with quoted spaces.
Author: Ryan Blue <blue@apache.org>
Closes#16107 from rdblue/SPARK-18677-fix-json-path.
### What changes were proposed in this pull request?
Added a test case for using joins with nested fields.
### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16110 from gatorsmile/followup-18674.
## What changes were proposed in this pull request?
Two bugs are addressed here
1. INSERT OVERWRITE TABLE sometime crashed when catalog partition management was enabled. This was because when dropping partitions after an overwrite operation, the Hive client will attempt to delete the partition files. If the entire partition directory was dropped, this would fail. The PR fixes this by adding a flag to control whether the Hive client should attempt to delete files.
2. The static partition spec for OVERWRITE TABLE was not correctly resolved to the case-sensitive original partition names. This resulted in the entire table being overwritten if you did not correctly capitalize your partition names.
cc yhuai cloud-fan
## How was this patch tested?
Unit tests. Surprisingly, the existing overwrite table tests did not catch these edge cases.
Author: Eric Liang <ekl@databricks.com>
Closes#16088 from ericl/spark-18659.
## What changes were proposed in this pull request?
This replaces uses of `TextOutputFormat` with an `OutputStream`, which will either write directly to the filesystem or indirectly via a compressor (if so configured). This avoids intermediate buffering.
The inverse of this (reading directly from a stream) is necessary for streaming large JSON records (when `wholeFile` is enabled) so I wanted to keep the read and write paths symmetric.
## How was this patch tested?
Existing unit tests.
Author: Nathan Howell <nhowell@godaddy.com>
Closes#16089 from NathanHowell/SPARK-18658.
## What changes were proposed in this pull request?
SPARK-18429 introduced count-min sketch aggregate function for SQL, but the implementation and testing is more complicated than needed. This simplifies the test cases and removes support for data types that don't have clear equality semantics:
1. Removed support for floating point and decimal types.
2. Removed the heavy randomized tests. The underlying CountMinSketch implementation already had pretty good test coverage through randomized tests, and the SPARK-18429 implementation is just to add an aggregate function wrapper around CountMinSketch. There is no need for randomized tests at three different levels of the implementations.
## How was this patch tested?
A lot of the change is to simplify test cases.
Author: Reynold Xin <rxin@databricks.com>
Closes#16093 from rxin/SPARK-18663.
## What changes were proposed in this pull request?
This PR makes `ExpressionEncoder.serializer.nullable` for flat encoder for a primitive type `false`. Since it is `true` for now, it is too conservative.
While `ExpressionEncoder.schema` has correct information (e.g. `<IntegerType, false>`), `serializer.head.nullable` of `ExpressionEncoder`, which got from `encoderFor[T]`, is always false. It is too conservative.
This is accomplished by checking whether a type is one of primitive types. If it is `true`, `nullable` should be `false`.
## How was this patch tested?
Added new tests for encoder and dataframe
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#15780 from kiszk/SPARK-18284.
## What changes were proposed in this pull request?
The current error message of USING join is quite confusing, for example:
```
scala> val df1 = List(1,2,3).toDS.withColumnRenamed("value", "c1")
df1: org.apache.spark.sql.DataFrame = [c1: int]
scala> val df2 = List(1,2,3).toDS.withColumnRenamed("value", "c2")
df2: org.apache.spark.sql.DataFrame = [c2: int]
scala> df1.join(df2, usingColumn = "c1")
org.apache.spark.sql.AnalysisException: using columns ['c1] can not be resolved given input columns: [c1, c2] ;;
'Join UsingJoin(Inner,List('c1))
:- Project [value#1 AS c1#3]
: +- LocalRelation [value#1]
+- Project [value#7 AS c2#9]
+- LocalRelation [value#7]
```
after this PR, it becomes:
```
scala> val df1 = List(1,2,3).toDS.withColumnRenamed("value", "c1")
df1: org.apache.spark.sql.DataFrame = [c1: int]
scala> val df2 = List(1,2,3).toDS.withColumnRenamed("value", "c2")
df2: org.apache.spark.sql.DataFrame = [c2: int]
scala> df1.join(df2, usingColumn = "c1")
org.apache.spark.sql.AnalysisException: USING column `c1` can not be resolved with the right join side, the right output is: [c2];
```
## How was this patch tested?
updated tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16100 from cloud-fan/natural.
## What changes were proposed in this pull request?
Due to confusion between URI vs paths, in certain cases we escape partition values too many times, which causes some Hive client operations to fail or write data to the wrong location. This PR fixes at least some of these cases.
To my understanding this is how values, filesystem paths, and URIs interact.
- Hive stores raw (unescaped) partition values that are returned to you directly when you call listPartitions.
- Internally, we convert these raw values to filesystem paths via `ExternalCatalogUtils.[un]escapePathName`.
- In some circumstances we store URIs instead of filesystem paths. When a path is converted to a URI via `path.toURI`, the escaped partition values are further URI-encoded. This means that to get a path back from a URI, you must call `new Path(new URI(uriTxt))` in order to decode the URI-encoded string.
- In `CatalogStorageFormat` we store URIs as strings. This makes it easy to forget to URI-decode the value before converting it into a path.
- Finally, the Hive client itself uses mostly Paths for representing locations, and only URIs occasionally.
In the future we should probably clean this up, perhaps by dropping use of URIs when unnecessary. We should also try fixing escaping for partition names as well as values, though names are unlikely to contain special characters.
cc mallman cloud-fan yhuai
## How was this patch tested?
Unit tests.
Author: Eric Liang <ekl@databricks.com>
Closes#16071 from ericl/spark-18635.
## What changes were proposed in this pull request?
For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL doesn't allow the entire row to be null, only its columns can be null. That's the reason we forbid users to use top level null objects in https://github.com/apache/spark/pull/13469
However, if users wrap non-flat type with `Option`, then we may still encoder top level null object to row, which is not allowed.
This PR fixes this case, and suggests users to wrap their type with `Tuple1` if they do wanna top level null objects.
## How was this patch tested?
new test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15979 from cloud-fan/option.
## What changes were proposed in this pull request?
Currently we haven't implemented `SHOW TABLE EXTENDED` in Spark 2.0. This PR is to implement the statement.
Goals:
1. Support `SHOW TABLES EXTENDED LIKE 'identifier_with_wildcards'`;
2. Explicitly output an unsupported error message for `SHOW TABLES [EXTENDED] ... PARTITION` statement;
3. Improve test cases for `SHOW TABLES` statement.
## How was this patch tested?
1. Add new test cases in file `show-tables.sql`.
2. Modify tests for `SHOW TABLES` in `DDLSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15958 from jiangxb1987/show-table-extended.
### What changes were proposed in this pull request?
The `constraints` of an operator is the expressions that evaluate to `true` for all the rows produced. That means, the expression result should be neither `false` nor `unknown` (NULL). Thus, we can conclude that `IsNotNull` on all the constraints, which are generated by its own predicates or propagated from the children. The constraint can be a complex expression. For better usage of these constraints, we try to push down `IsNotNull` to the lowest-level expressions (i.e., `Attribute`). `IsNotNull` can be pushed through an expression when it is null intolerant. (When the input is NULL, the null-intolerant expression always evaluates to NULL.)
Below is the existing code we have for `IsNotNull` pushdown.
```Scala
private def scanNullIntolerantExpr(expr: Expression): Seq[Attribute] = expr match {
case a: Attribute => Seq(a)
case _: NullIntolerant | IsNotNull(_: NullIntolerant) =>
expr.children.flatMap(scanNullIntolerantExpr)
case _ => Seq.empty[Attribute]
}
```
**`IsNotNull` itself is not null-intolerant.** It converts `null` to `false`. If the expression does not include any `Not`-like expression, it works; otherwise, it could generate a wrong result. This PR is to fix the above function by removing the `IsNotNull` from the inference. After the fix, when a constraint has a `IsNotNull` expression, we infer new attribute-specific `IsNotNull` constraints if and only if `IsNotNull` appears in the root.
Without the fix, the following test case will return empty.
```Scala
val data = Seq[java.lang.Integer](1, null).toDF("key")
data.filter("not key is not null").show()
```
Before the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter (isnotnull(value#1) && NOT isnotnull(value#1))
+- LocalRelation [value#1]
```
After the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter NOT isnotnull(value#1)
+- LocalRelation [value#1]
```
### How was this patch tested?
Added a test
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16067 from gatorsmile/isNotNull2.
## What changes were proposed in this pull request?
The result of a `sum` aggregate function is typically a Decimal, Double or a Long. Currently the output dataType is based on input's dataType.
The `FunctionArgumentConversion` rule will make sure that the input is promoted to the largest type, and that also ensures that the output uses a (hopefully) sufficiently large output dataType. The issue is that sum is in a resolved state when we cast the input type, this means that rules assuming that the dataType of the expression does not change anymore could have been applied in the mean time. This is what happens if we apply `WidenSetOperationTypes` before applying the casts, and this breaks analysis.
The most straight forward and future proof solution is to make `sum` always output the widest dataType in its class (Long for IntegralTypes, Decimal for DecimalTypes & Double for FloatType and DoubleType). This PR implements that solution.
We should move expression specific type casting rules into the given Expression at some point.
## How was this patch tested?
Added (regression) tests to SQLQueryTestSuite's `union.sql`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16063 from hvanhovell/SPARK-18622.
## What changes were proposed in this pull request?
`AggregateFunction` currently implements `ImplicitCastInputTypes` (which enables implicit input type casting). There are actually quite a few situations in which we don't need this, or require more control over our input. A recent example is the aggregate for `CountMinSketch` which should only take string, binary or integral types inputs.
This PR removes `ImplicitCastInputTypes` from the `AggregateFunction` and makes a case-by-case decision on what kind of input validation we should use.
## How was this patch tested?
Refactoring only. Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16066 from hvanhovell/SPARK-18632.
## What changes were proposed in this pull request?
ExistenceJoin should be treated the same as LeftOuter and LeftAnti, not InnerLike and LeftSemi. This is not currently exposed because the rewrite of [NOT] EXISTS OR ... to ExistenceJoin happens in rule RewritePredicateSubquery, which is in a separate rule set and placed after the rule PushPredicateThroughJoin. During the transformation in the rule PushPredicateThroughJoin, an ExistenceJoin never exists.
The semantics of ExistenceJoin says we need to preserve all the rows from the left table through the join operation as if it is a regular LeftOuter join. The ExistenceJoin augments the LeftOuter operation with a new column called exists, set to true when the join condition in the ON clause is true and false otherwise. The filter of any rows will happen in the Filter operation above the ExistenceJoin.
Example:
A(c1, c2): { (1, 1), (1, 2) }
// B can be any value as it is irrelevant in this example
B(c1): { (NULL) }
select A.*
from A
where exists (select 1 from B where A.c1 = A.c2)
or A.c2=2
In this example, the correct result is all the rows from A. If the pattern ExistenceJoin around line 935 in Optimizer.scala is indeed active, the code will push down the predicate A.c1 = A.c2 to be a Filter on relation A, which will incorrectly filter the row (1,2) from A.
## How was this patch tested?
Since this is not an exposed case, no new test cases is added. The scenario is discovered via a code review of another PR and confirmed to be valid with peer.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#16044 from nsyca/spark-18614.
## What changes were proposed in this pull request?
This PR implements a new Aggregate to generate count min sketch, which is a wrapper of CountMinSketch.
## How was this patch tested?
add test cases
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#15877 from wzhfy/cms.
## What changes were proposed in this pull request?
This PR make `sbt unidoc` complete with Java 8.
This PR roughly includes several fixes as below:
- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``
```diff
- * A column that will be computed based on the data in a [[DataFrame]].
+ * A column that will be computed based on the data in a `DataFrame`.
```
- Fix throws annotations so that they are recognisable in javadoc
- Fix URL links to `<a href="http..."></a>`.
```diff
- * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression.
+ * <a href="http://en.wikipedia.org/wiki/Decision_tree_learning">
+ * Decision tree (Wikipedia)</a> model for regression.
```
```diff
- * see http://en.wikipedia.org/wiki/Receiver_operating_characteristic
+ * see <a href="http://en.wikipedia.org/wiki/Receiver_operating_characteristic">
+ * Receiver operating characteristic (Wikipedia)</a>
```
- Fix < to > to
- `greater than`/`greater than or equal to` or `less than`/`less than or equal to` where applicable.
- Wrap it with `{{{...}}}` to print them in javadoc or use `{code ...}` or `{literal ..}`. Please refer https://github.com/apache/spark/pull/16013#discussion_r89665558
- Fix `</p>` complaint
## How was this patch tested?
Manually tested by `jekyll build` with Java 7 and 8
```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```
```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16013 from HyukjinKwon/SPARK-3359-errors-more.
## What changes were proposed in this pull request?
For the following workflow:
1. I have a column called time which is at minute level precision in a Streaming DataFrame
2. I want to perform groupBy time, count
3. Then I want my MemorySink to only have the last 30 minutes of counts and I perform this by
.where('time >= current_timestamp().cast("long") - 30 * 60)
what happens is that the `filter` gets pushed down before the aggregation, and the filter happens on the source data for the aggregation instead of the result of the aggregation (where I actually want to filter).
I guess the main issue here is that `current_timestamp` is non-deterministic in the streaming context and shouldn't be pushed down the filter.
Does this require us to store the `current_timestamp` for each trigger of the streaming job, that is something to discuss.
Furthermore, we want to persist current batch timestamp and watermark timestamp to the offset log so that these values are consistent across multiple executions of the same batch.
brkyvz zsxwing tdas
## How was this patch tested?
A test was added to StreamingAggregationSuite ensuring the above use case is handled. The test injects a stream of time values (in seconds) to a query that runs in complete mode and only outputs the (count) aggregation results for the past 10 seconds.
Author: Tyson Condie <tcondie@gmail.com>
Closes#15949 from tcondie/SPARK-18339.
## What changes were proposed in this pull request?
This is absolutely minor. PR https://github.com/apache/spark/pull/15595 uses `dt1.asNullable == dt2.asNullable` expressions in a few places. It is however more efficient to call `dt1.sameType(dt2)`. I have replaced every instance of the first pattern with the second pattern (3/5 were introduced by #15595).
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16041 from hvanhovell/SPARK-18058.
## What changes were proposed in this pull request?
In #15764 we added a mechanism to detect if a function is temporary or not. Hive functions are treated as non-temporary. Of the three hive functions, now "percentile" has been implemented natively, and "hash" has been removed. So we should update the list.
## How was this patch tested?
Unit tests.
Author: Shuai Lin <linshuai2012@gmail.com>
Closes#16049 from lins05/update-temp-function-detect-hive-list.
## What changes were proposed in this pull request?
Implement percentile SQL function. It computes the exact percentile(s) of expr at pc with range in [0, 1].
## How was this patch tested?
Add a new testsuite `PercentileSuite` to test percentile directly.
Updated related testcases in `ExpressionToSQLSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Author: 蒋星博 <jiangxingbo@meituan.com>
Author: jiangxingbo <jiangxingbo@meituan.com>
Closes#14136 from jiangxb1987/percentile.
## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/15704 will fail if we use int literal in `DROP PARTITION`, and we have reverted it in branch-2.1.
This PR reverts it in master branch, and add a regression test for it, to make sure the master branch is healthy.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16036 from cloud-fan/revert.
## What changes were proposed in this pull request?
We currently push down join conditions of a Left Anti join to both sides of the join. This is similar to Inner, Left Semi and Existence (a specialized left semi) join. The problem is that this changes the semantics of the join; a left anti join filters out rows that matches the join condition.
This PR fixes this by only pushing down conditions to the left hand side of the join. This is similar to the behavior of left outer join.
## How was this patch tested?
Added tests to `FilterPushdownSuite.scala` and created a SQLQueryTestSuite file for left anti joins with a regression test.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16026 from hvanhovell/SPARK-18597.
## What changes were proposed in this pull request?
The `CollapseWindow` optimizer rule changes the order of output attributes. This modifies the output of the plan, which the optimizer cannot do. This also breaks things like `collect()` for which we use a `RowEncoder` that assumes that the output attributes of the executed plan are equal to those outputted by the logical plan.
## How was this patch tested?
I have updated an incorrect test in `CollapseWindowSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#16027 from hvanhovell/SPARK-18604.
## What changes were proposed in this pull request?
Janino can optimize `true ? a : b` into `a` or `false ? a : b` into `b`, or if/else with literal condition, so we should use literal as `ev.isNull` if possible.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#16008 from ueshin/issues/SPARK-18585.
### What changes were proposed in this pull request?
Currently, the name validation checks are limited to table creation. It is enfored by Analyzer rule: `PreWriteCheck`.
However, table renaming and database creation have the same issues. It makes more sense to do the checks in `SessionCatalog`. This PR is to add it into `SessionCatalog`.
### How was this patch tested?
Added test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#16018 from gatorsmile/nameValidate.
## What changes were proposed in this pull request?
Currently, `OuterReference` is not `NamedExpression`. So, it raises 'ClassCastException` when it used in projection lists of IN correlated subqueries. This PR aims to support that by making `OuterReference` as `NamedExpression` to show correct error messages.
```scala
scala> sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES 1, 2 AS t1(a)")
scala> sql("CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES 1 AS t2(b)")
scala> sql("SELECT a FROM t1 WHERE a IN (SELECT a FROM t2)").show
java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.OuterReference cannot be cast to org.apache.spark.sql.catalyst.expressions.NamedExpression
```
## How was this patch tested?
Pass the Jenkins test with new test cases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#16015 from dongjoon-hyun/SPARK-17251-2.
## What changes were proposed in this pull request?
The nullability of `InputFileName` should be `false`.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#16007 from ueshin/issues/SPARK-18583.
## What changes were proposed in this pull request?
The expression `in(empty seq)` is invalid in some data source. Since `in(empty seq)` is always false, we should generate `in(empty seq)` to false literal in optimizer.
The sql `SELECT * FROM t WHERE a IN ()` throws a `ParseException` which is consistent with Hive, don't need to change that behavior.
## How was this patch tested?
Add new test case in `OptimizeInSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15977 from jiangxb1987/isin-empty.
## What changes were proposed in this pull request?
In `HyperLogLogPlusPlus`, if the relative error is so small that p >= 19, it will cause ArrayIndexOutOfBoundsException in `THRESHOLDS(p-4)` . We should check `p` and when p >= 19, regress to the original HLL result and use the small range correction they use.
The pr also fixes the upper bound in the log info in `require()`.
The upper bound is computed by:
```
val relativeSD = 1.106d / Math.pow(Math.E, p * Math.log(2.0d) / 2.0d)
```
which is derived from the equation for computing `p`:
```
val p = 2.0d * Math.log(1.106d / relativeSD) / Math.log(2.0d)
```
## How was this patch tested?
add test cases for:
1. checking validity of parameter relatvieSD
2. estimation with smaller relative error so that p >= 19
Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#15990 from wzhfy/hllppRsd.
## What changes were proposed in this pull request?
This PR only tries to fix things that looks pretty straightforward and were fixed in other previous PRs before.
This PR roughly fixes several things as below:
- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``
```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/DataStreamReader.java:226: error: reference not found
[error] * Loads text files and returns a {link DataFrame} whose schema starts with a string column named
```
- Fix an exception annotation and remove code backticks in `throws` annotation
Currently, sbt unidoc with Java 8 complains as below:
```
[error] .../java/org/apache/spark/sql/streaming/StreamingQuery.java:72: error: unexpected text
[error] * throws StreamingQueryException, if <code>this</code> query has terminated with an exception.
```
`throws` should specify the correct class name from `StreamingQueryException,` to `StreamingQueryException` without backticks. (see [JDK-8007644](https://bugs.openjdk.java.net/browse/JDK-8007644)).
- Fix `[[http..]]` to `<a href="http..."></a>`.
```diff
- * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle
- * blog page]].
+ * <a href="https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https">
+ * Oracle blog page</a>.
```
`[[http...]]` link markdown in scaladoc is unrecognisable in javadoc.
- It seems class can't have `return` annotation. So, two cases of this were removed.
```
[error] .../java/org/apache/spark/mllib/regression/IsotonicRegression.java:27: error: invalid use of return
[error] * return New instance of IsotonicRegression.
```
- Fix < to `<` and > to `>` according to HTML rules.
- Fix `</p>` complaint
- Exclude unrecognisable in javadoc, `constructor`, `todo` and `groupname`.
## How was this patch tested?
Manually tested by `jekyll build` with Java 7 and 8
```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```
```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```
Note: this does not yet make sbt unidoc suceed with Java 8 yet but it reduces the number of errors with Java 8.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15999 from HyukjinKwon/SPARK-3359-errors.
## What changes were proposed in this pull request?
- Raise Analysis exception when correlated predicates exist in the descendant operators of either operand of a Full outer join in a subquery as well as in a FOJ operator itself
- Raise Analysis exception when correlated predicates exists in a Window operator (a side effect inadvertently introduced by SPARK-17348)
## How was this patch tested?
Run sql/test catalyst/test and new test cases, added to SubquerySuite, showing the reported incorrect results.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#16005 from nsyca/FOJ-incorrect.1.
## What changes were proposed in this pull request?
The current implementation of column stats uses the base64 encoding of the internal UnsafeRow format to persist statistics (in table properties in Hive metastore). This is an internal format that is not stable across different versions of Spark and should NOT be used for persistence. In addition, it would be better if statistics stored in the catalog is human readable.
This pull request introduces the following changes:
1. Created a single ColumnStat class to for all data types. All data types track the same set of statistics.
2. Updated the implementation for stats collection to get rid of the dependency on internal data structures (e.g. InternalRow, or storing DateType as an int32). For example, previously dates were stored as a single integer, but are now stored as java.sql.Date. When we implement the next steps of CBO, we can add code to convert those back into internal types again.
3. Documented clearly what JVM data types are being used to store what data.
4. Defined a simple Map[String, String] interface for serializing and deserializing column stats into/from the catalog.
5. Rearranged the method/function structure so it is more clear what the supported data types are, and also moved how stats are generated into ColumnStat class so they are easy to find.
## How was this patch tested?
Removed most of the original test cases created for column statistics, and added three very simple ones to cover all the cases. The three test cases validate:
1. Roundtrip serialization works.
2. Behavior when analyzing non-existent column or unsupported data type column.
3. Result for stats collection for all valid data types.
Also moved parser related tests into a parser test suite and added an explicit serialization test for the Hive external catalog.
Author: Reynold Xin <rxin@databricks.com>
Closes#15959 from rxin/SPARK-18522.
## What changes were proposed in this pull request?
In Spark SQL, some expression may output safe format values, e.g. `CreateArray`, `CreateStruct`, `Cast`, etc. When we compare 2 values, we should be able to compare safe and unsafe formats.
The `GreaterThan`, `LessThan`, etc. in Spark SQL already handles it, but the `EqualTo` doesn't. This PR fixes it.
## How was this patch tested?
new unit test and regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15929 from cloud-fan/type-aware.
## What changes were proposed in this pull request?
This PR proposes throwing an `AnalysisException` with a proper message rather than `NoSuchElementException` with the message ` key not found: TimestampType` when unsupported types are given to `reflect` and `java_method` functions.
```scala
spark.range(1).selectExpr("reflect('java.lang.String', 'valueOf', cast('1990-01-01' as timestamp))")
```
produces
**Before**
```
java.util.NoSuchElementException: key not found: TimestampType
at scala.collection.MapLike$class.default(MapLike.scala:228)
at scala.collection.AbstractMap.default(Map.scala:59)
at scala.collection.MapLike$class.apply(MapLike.scala:141)
at scala.collection.AbstractMap.apply(Map.scala:59)
at org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection$$anonfun$findMethod$1$$anonfun$apply$1.apply(CallMethodViaReflection.scala:159)
...
```
**After**
```
cannot resolve 'reflect('java.lang.String', 'valueOf', CAST('1990-01-01' AS TIMESTAMP))' due to data type mismatch: arguments from the third require boolean, byte, short, integer, long, float, double or string expressions; line 1 pos 0;
'Project [unresolvedalias(reflect(java.lang.String, valueOf, cast(1990-01-01 as timestamp)), Some(<function1>))]
+- Range (0, 1, step=1, splits=Some(2))
...
```
Added message is,
```
arguments from the third require boolean, byte, short, integer, long, float, double or string expressions
```
## How was this patch tested?
Tests added in `CallMethodViaReflection`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15694 from HyukjinKwon/SPARK-18179.
## What changes were proposed in this pull request?
Fixes the inconsistency of error raised between data source and hive serde
tables when schema is specified in CTAS scenario. In the process the grammar for
create table (datasource) is simplified.
**before:**
``` SQL
spark-sql> create table t2 (c1 int, c2 int) using parquet as select * from t1;
Error in query:
mismatched input 'as' expecting {<EOF>, '.', 'OPTIONS', 'CLUSTERED', 'PARTITIONED'}(line 1, pos 64)
== SQL ==
create table t2 (c1 int, c2 int) using parquet as select * from t1
----------------------------------------------------------------^^^
```
**After:**
```SQL
spark-sql> create table t2 (c1 int, c2 int) using parquet as select * from t1
> ;
Error in query:
Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement(line 1, pos 0)
== SQL ==
create table t2 (c1 int, c2 int) using parquet as select * from t1
^^^
```
## How was this patch tested?
Added a new test in CreateTableAsSelectSuite
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#15968 from dilipbiswal/ctas.
## What changes were proposed in this pull request?
While this behavior is debatable, consider the following use case:
```sql
UNCACHE TABLE foo;
CACHE TABLE foo AS
SELECT * FROM bar
```
The command above fails the first time you run it. But I want to run the command above over and over again, and I don't want to change my code just for the first run of it.
The issue is that subsequent `CACHE TABLE` commands do not overwrite the existing table.
Now we can do:
```sql
UNCACHE TABLE IF EXISTS foo;
CACHE TABLE foo AS
SELECT * FROM bar
```
## How was this patch tested?
Unit tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#15896 from brkyvz/uncache.
## What changes were proposed in this pull request?
This PR blocks an incorrect result scenario in scalar subquery where there are GROUP BY column(s)
that are not part of the correlated predicate(s).
Example:
// Incorrect result
Seq(1).toDF("c1").createOrReplaceTempView("t1")
Seq((1,1),(1,2)).toDF("c1","c2").createOrReplaceTempView("t2")
sql("select (select sum(-1) from t2 where t1.c1=t2.c1 group by t2.c2) from t1").show
// How can selecting a scalar subquery from a 1-row table return 2 rows?
## How was this patch tested?
sql/test, catalyst/test
new test case covering the reported problem is added to SubquerySuite.scala
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#15936 from nsyca/scalarSubqueryIncorrect-1.
## What changes were proposed in this pull request?
Technically map type is not orderable, but can be used in equality comparison. However, due to the limitation of the current implementation, map type can't be used in equality comparison so that it can't be join key or grouping key.
This PR makes this limitation explicit, to avoid wrong result.
## How was this patch tested?
updated tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15956 from cloud-fan/map-type.
## What changes were proposed in this pull request?
The nullabilities of `MapObject` can be made more strict by relying on `inputObject.nullable` and `lambdaFunction.nullable`.
Also `ExternalMapToCatalyst.dataType` can be made more strict by relying on `valueConverter.nullable`.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#15840 from ueshin/issues/SPARK-18398.
## What changes were proposed in this pull request?
This pr extracts method for preparing arguments from `StaticInvoke`, `Invoke` and `NewInstance` and modify to short circuit if arguments have `null` when `propageteNull == true`.
The steps are as follows:
1. Introduce `InvokeLike` to extract common logic from `StaticInvoke`, `Invoke` and `NewInstance` to prepare arguments.
`StaticInvoke` and `Invoke` had a risk to exceed 64kb JVM limit to prepare arguments but after this patch they can handle them because they share the preparing code of NewInstance, which handles the limit well.
2. Remove unneeded null checking and fix nullability of `NewInstance`.
Avoid some of nullabilty checking which are not needed because the expression is not nullable.
3. Modify to short circuit if arguments have `null` when `needNullCheck == true`.
If `needNullCheck == true`, preparing arguments can be skipped if we found one of them is `null`, so modified to short circuit in the case.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#15901 from ueshin/issues/SPARK-18467.
## What changes were proposed in this pull request?
This PR adds code generation to `Generate`. It supports two code paths:
- General `TraversableOnce` based iteration. This used for regular `Generator` (code generation supporting) expressions. This code path expects the expression to return a `TraversableOnce[InternalRow]` and it will iterate over the returned collection. This PR adds code generation for the `stack` generator.
- Specialized `ArrayData/MapData` based iteration. This is used for the `explode`, `posexplode` & `inline` functions and operates directly on the `ArrayData`/`MapData` result that the child of the generator returns.
### Benchmarks
I have added some benchmarks and it seems we can create a nice speedup for explode:
#### Environment
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6
Intel(R) Core(TM) i7-4980HQ CPU 2.80GHz
```
#### Explode Array
##### Before
```
generate explode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off 7377 / 7607 2.3 439.7 1.0X
generate explode array wholestage on 6055 / 6086 2.8 360.9 1.2X
```
##### After
```
generate explode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off 7432 / 7696 2.3 443.0 1.0X
generate explode array wholestage on 631 / 646 26.6 37.6 11.8X
```
#### Explode Map
##### Before
```
generate explode map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off 12792 / 12848 1.3 762.5 1.0X
generate explode map wholestage on 11181 / 11237 1.5 666.5 1.1X
```
##### After
```
generate explode map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off 10949 / 10972 1.5 652.6 1.0X
generate explode map wholestage on 870 / 913 19.3 51.9 12.6X
```
#### Posexplode
##### Before
```
generate posexplode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off 7547 / 7580 2.2 449.8 1.0X
generate posexplode array wholestage on 5786 / 5838 2.9 344.9 1.3X
```
##### After
```
generate posexplode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off 7535 / 7548 2.2 449.1 1.0X
generate posexplode array wholestage on 620 / 624 27.1 37.0 12.1X
```
#### Inline
##### Before
```
generate inline array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off 6935 / 6978 2.4 413.3 1.0X
generate inline array wholestage on 6360 / 6400 2.6 379.1 1.1X
```
##### After
```
generate inline array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off 6940 / 6966 2.4 413.6 1.0X
generate inline array wholestage on 1002 / 1012 16.7 59.7 6.9X
```
#### Stack
##### Before
```
generate stack: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off 12980 / 13104 1.3 773.7 1.0X
generate stack wholestage on 11566 / 11580 1.5 689.4 1.1X
```
##### After
```
generate stack: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off 12875 / 12949 1.3 767.4 1.0X
generate stack wholestage on 840 / 845 20.0 50.0 15.3X
```
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Herman van Hovell <hvanhovell@questtec.nl>
Closes#13065 from hvanhovell/SPARK-15214.
## What changes were proposed in this pull request?
The previous documentation and example for DateDiff was wrong.
## How was this patch tested?
Doc only change.
Author: Reynold Xin <rxin@databricks.com>
Closes#15937 from rxin/datediff-doc.
## What changes were proposed in this pull request?
The nullability of `WrapOption` should be `false`.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#15887 from ueshin/issues/SPARK-18442.
### What changes were proposed in this pull request?
Currently, when CTE is used in RunnableCommand, the Analyzer does not replace the logical node `With`. The child plan of RunnableCommand is not resolved. Thus, the output of the `With` plan node looks very confusing.
For example,
```
sql(
"""
|CREATE VIEW cte_view AS
|WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
|SELECT n FROM w
""".stripMargin).explain()
```
The output is like
```
ExecutedCommand
+- CreateViewCommand `cte_view`, WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
SELECT n FROM w, false, false, PersistedView
+- 'With [(w,SubqueryAlias w
+- Project [1 AS n#16]
+- OneRowRelation$
), (cte1,'SubqueryAlias cte1
+- 'Project [unresolvedalias(2, None)]
+- OneRowRelation$
), (cte2,'SubqueryAlias cte2
+- 'Project [unresolvedalias(3, None)]
+- OneRowRelation$
)]
+- 'Project ['n]
+- 'UnresolvedRelation `w`
```
After the fix, the output is as shown below.
```
ExecutedCommand
+- CreateViewCommand `cte_view`, WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
SELECT n FROM w, false, false, PersistedView
+- CTE [w, cte1, cte2]
: :- SubqueryAlias w
: : +- Project [1 AS n#16]
: : +- OneRowRelation$
: :- 'SubqueryAlias cte1
: : +- 'Project [unresolvedalias(2, None)]
: : +- OneRowRelation$
: +- 'SubqueryAlias cte2
: +- 'Project [unresolvedalias(3, None)]
: +- OneRowRelation$
+- 'Project ['n]
+- 'UnresolvedRelation `w`
```
BTW, this PR also fixes the output of the view type.
### How was this patch tested?
Manual
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15854 from gatorsmile/cteName.
## What changes were proposed in this pull request?
Small fix, fix the errors caused by lint check in Java
- Clear unused objects and `UnusedImports`.
- Add comments around the method `finalize` of `NioBufferedFileInputStream`to turn off checkstyle.
- Cut the line which is longer than 100 characters into two lines.
## How was this patch tested?
Travis CI.
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
```
Before:
```
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[21,8] (imports) UnusedImports: Unused import - org.apache.commons.crypto.cipher.CryptoCipherFactory.
[ERROR] src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java:[516,5] (modifier) RedundantModifier: Redundant 'public' modifier.
[ERROR] src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java:[133] (coding) NoFinalizer: Avoid using finalizer method.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java:[71] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java:[112] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[31,17] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
[ERROR]src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java:[64] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.ml.linalg.Vectors.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
```
After:
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
Using `mvn` from path: /home/travis/build/ConeyLiu/spark/build/apache-maven-3.3.9/bin/mvn
Checkstyle checks passed.
```
Author: Xianyang Liu <xyliu0530@icloud.com>
Closes#15865 from ConeyLiu/master.
## What changes were proposed in this pull request?
This PR aims to improve DataSource option keys to be more case-insensitive
DataSource partially use CaseInsensitiveMap in code-path. For example, the following fails to find url.
```scala
val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2)
df.write.format("jdbc")
.option("UrL", url1)
.option("dbtable", "TEST.SAVETEST")
.options(properties.asScala)
.save()
```
This PR makes DataSource options to use CaseInsensitiveMap internally and also makes DataSource to use CaseInsensitiveMap generally except `InMemoryFileIndex` and `InsertIntoHadoopFsRelationCommand`. We can not pass them CaseInsensitiveMap because they creates new case-sensitive HadoopConfs by calling newHadoopConfWithOptions(options) inside.
## How was this patch tested?
Pass the Jenkins test with newly added test cases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#15884 from dongjoon-hyun/SPARK-18433.
## What changes were proposed in this pull request?
it's weird that every session can set its own warehouse path at runtime, we should forbid it and make it a static conf.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15825 from cloud-fan/warehouse.
## What changes were proposed in this pull request?
Commit f14ae4900a broke the scala 2.10 build. This PR fixes this by simplifying the used pattern match.
## How was this patch tested?
Tested building manually. Ran `build/sbt -Dscala-2.10 -Pscala-2.10 package`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15891 from hvanhovell/SPARK-18300-scala-2.10.
## What changes were proposed in this pull request?
The `FoldablePropagation` optimizer rule, pulls foldable values out from under an `Expand`. This breaks the `Expand` in two ways:
- It rewrites the output attributes of the `Expand`. We explicitly define output attributes for `Expand`, these are (unfortunately) considered as part of the expressions of the `Expand` and can be rewritten.
- Expand can actually change the column (it will typically re-use the attributes or the underlying plan). This means that we cannot safely propagate the expressions from under an `Expand`.
This PR fixes this and (hopefully) other issues by explicitly whitelisting allowed operators.
## How was this patch tested?
Added tests to `FoldablePropagationSuite` and to `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15857 from hvanhovell/SPARK-18300.
### What changes were proposed in this pull request?
When the exception is an invocation exception during function lookup, we return a useless/confusing error message:
For example,
```Scala
df.selectExpr("concat_ws()")
```
Below is the error message we got:
```
null; line 1 pos 0
org.apache.spark.sql.AnalysisException: null; line 1 pos 0
```
To get the meaningful error message, we need to get the cause. The fix is exactly the same as what we did in https://github.com/apache/spark/pull/12136. After the fix, the message we got is the exception issued in the constuctor of function implementation:
```
requirement failed: concat_ws requires at least one argument.; line 1 pos 0
org.apache.spark.sql.AnalysisException: requirement failed: concat_ws requires at least one argument.; line 1 pos 0
```
### How was this patch tested?
Added test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15878 from gatorsmile/functionNotFound.
This PR adds a new method `withWatermark` to the `Dataset` API, which can be used specify an _event time watermark_. An event time watermark allows the streaming engine to reason about the point in time after which we no longer expect to see late data. This PR also has augmented `StreamExecution` to use this watermark for several purposes:
- To know when a given time window aggregation is finalized and thus results can be emitted when using output modes that do not allow updates (e.g. `Append` mode).
- To minimize the amount of state that we need to keep for on-going aggregations, by evicting state for groups that are no longer expected to change. Although, we do still maintain all state if the query requires (i.e. if the event time is not present in the `groupBy` or when running in `Complete` mode).
An example that emits windowed counts of records, waiting up to 5 minutes for late data to arrive.
```scala
df.withWatermark("eventTime", "5 minutes")
.groupBy(window($"eventTime", "1 minute") as 'window)
.count()
.writeStream
.format("console")
.mode("append") // In append mode, we only output finalized aggregations.
.start()
```
### Calculating the watermark.
The current event time is computed by looking at the `MAX(eventTime)` seen this epoch across all of the partitions in the query minus some user defined _delayThreshold_. An additional constraint is that the watermark must increase monotonically.
Note that since we must coordinate this value across partitions occasionally, the actual watermark used is only guaranteed to be at least `delay` behind the actual event time. In some cases we may still process records that arrive more than delay late.
This mechanism was chosen for the initial implementation over processing time for two reasons:
- it is robust to downtime that could affect processing delay
- it does not require syncing of time or timezones between the producer and the processing engine.
### Other notable implementation details
- A new trigger metric `eventTimeWatermark` outputs the current value of the watermark.
- We mark the event time column in the `Attribute` metadata using the key `spark.watermarkDelay`. This allows downstream operations to know which column holds the event time. Operations like `window` propagate this metadata.
- `explain()` marks the watermark with a suffix of `-T${delayMs}` to ease debugging of how this information is propagated.
- Currently, we don't filter out late records, but instead rely on the state store to avoid emitting records that are both added and filtered in the same epoch.
### Remaining in this PR
- [ ] The test for recovery is currently failing as we don't record the watermark used in the offset log. We will need to do so to ensure determinism, but this is deferred until #15626 is merged.
### Other follow-ups
There are some natural additional features that we should consider for future work:
- Ability to write records that arrive too late to some external store in case any out-of-band remediation is required.
- `Update` mode so you can get partial results before a group is evicted.
- Other mechanisms for calculating the watermark. In particular a watermark based on quantiles would be more robust to outliers.
Author: Michael Armbrust <michael@databricks.com>
Closes#15702 from marmbrus/watermarks.
## What changes were proposed in this pull request?
Return an Analysis exception when there is a correlated non-equality predicate in a subquery and the correlated column from the outer reference is not from the immediate parent operator of the subquery. This PR prevents incorrect results from subquery transformation in such case.
Test cases, both positive and negative tests, are added.
## How was this patch tested?
sql/test, catalyst/test, hive/test, and scenarios that will produce incorrect results without this PR and product correct results when subquery transformation does happen.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#15763 from nsyca/spark-17348.
## What changes were proposed in this pull request?
This removes the serialization test from RegexpExpressionsSuite and
replaces it by serializing all expressions in checkEvaluation.
This also fixes math constant expressions by making LeafMathExpression
Serializable and fixes NumberFormat values that are null or invalid
after serialization.
## How was this patch tested?
This patch is to tests.
Author: Ryan Blue <blue@apache.org>
Closes#15847 from rdblue/SPARK-18387-fix-serializable-expressions.
## What changes were proposed in this pull request?
As of current 2.1, INSERT OVERWRITE with dynamic partitions against a Datasource table will overwrite the entire table instead of only the partitions matching the static keys, as in Hive. It also doesn't respect custom partition locations.
This PR adds support for all these operations to Datasource tables managed by the Hive metastore. It is implemented as follows
- During planning time, the full set of partitions affected by an INSERT or OVERWRITE command is read from the Hive metastore.
- The planner identifies any partitions with custom locations and includes this in the write task metadata.
- FileFormatWriter tasks refer to this custom locations map when determining where to write for dynamic partition output.
- When the write job finishes, the set of written partitions is compared against the initial set of matched partitions, and the Hive metastore is updated to reflect the newly added / removed partitions.
It was necessary to introduce a method for staging files with absolute output paths to `FileCommitProtocol`. These files are not handled by the Hadoop output committer but are moved to their final locations when the job commits.
The overwrite behavior of legacy Datasource tables is also changed: no longer will the entire table be overwritten if a partial partition spec is present.
cc cloud-fan yhuai
## How was this patch tested?
Unit tests, existing tests.
Author: Eric Liang <ekl@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15814 from ericl/sc-5027.
## What changes were proposed in this pull request?
This PR corrects several partition related behaviors of `ExternalCatalog`:
1. default partition location should not always lower case the partition column names in path string(fix `HiveExternalCatalog`)
2. rename partition should not always lower case the partition column names in updated partition path string(fix `HiveExternalCatalog`)
3. rename partition should update the partition location only for managed table(fix `InMemoryCatalog`)
4. create partition with existing directory should be fine(fix `InMemoryCatalog`)
5. create partition with non-existing directory should create that directory(fix `InMemoryCatalog`)
6. drop partition from external table should not delete the directory(fix `InMemoryCatalog`)
## How was this patch tested?
new tests in `ExternalCatalogSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15797 from cloud-fan/partition.
## What changes were proposed in this pull request?
This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.
## How was this patch tested?
* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.
Author: Ryan Blue <blue@apache.org>
Closes#15834 from rdblue/SPARK-18368-fix-regexp-replace.
## What changes were proposed in this pull request?
This makes the result value both transient and lazy, so that if the RegExpReplace object is initialized then serialized, `result: StringBuffer` will be correctly initialized.
## How was this patch tested?
* Verified that this patch fixed the query that found the bug.
* Added a test case that fails without the fix.
Author: Ryan Blue <blue@apache.org>
Closes#15816 from rdblue/SPARK-18368-fix-regexp-replace.
## What changes were proposed in this pull request?
We generate bitmasks for grouping sets during the parsing process, and use these during analysis. These bitmasks are difficult to work with in practice and have lead to numerous bugs. This PR removes these and use actual sets instead, however we still need to generate these offsets for the grouping_id.
This PR does the following works:
1. Replace bitmasks by actual grouping sets durning Parsing/Analysis stage of CUBE/ROLLUP/GROUPING SETS;
2. Add new testsuite `ResolveGroupingAnalyticsSuite` to test the `Analyzer.ResolveGroupingAnalytics` rule directly;
3. Fix a minor bug in `ResolveGroupingAnalytics`.
## How was this patch tested?
By existing test cases, and add new testsuite `ResolveGroupingAnalyticsSuite` to test directly.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15484 from jiangxb1987/group-set.
## What changes were proposed in this pull request?
In RewriteDistinctAggregates rewrite funtion,after the UDAF's childs are mapped to AttributeRefference, If the UDAF(such as ApproximatePercentile) has a foldable TypeCheck for the input, It will failed because the AttributeRefference is not foldable,then the UDAF is not resolved, and then nullify on the unresolved object will throw a Exception.
In this PR, only map Unfoldable child to AttributeRefference, this can avoid the UDAF's foldable TypeCheck. and then only Expand Unfoldable child, there is no need to Expand a static value(foldable value).
**Before sql result**
> select percentile_approxy(key,0.99999),count(distinct key),sume(distinc key) from src limit 1
> org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to dataType on unresolved object, tree: 'percentile_approx(CAST(src.`key` AS DOUBLE), CAST(0.99999BD AS DOUBLE), 10000)
> at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:92)
> at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$.org$apache$spark$sql$catalyst$optimizer$RewriteDistinctAggregates$$nullify(RewriteDistinctAggregates.scala:261)
**After sql result**
> select percentile_approxy(key,0.99999),count(distinct key),sume(distinc key) from src limit 1
> [498.0,309,79136]
## How was this patch tested?
Add a test case in HiveUDFSuit.
Author: root <root@iZbp1gsnrlfzjxh82cz80vZ.(none)>
Closes#15668 from windpiger/RewriteDistinctUDAFUnresolveExcep.
### What changes were proposed in this pull request?
Based on the discussion in [SPARK-18209](https://issues.apache.org/jira/browse/SPARK-18209). It doesn't really make sense to create permanent views based on temporary views or temporary UDFs.
To disallow the supports and issue the exceptions, this PR needs to detect whether a temporary view/UDF is being used when defining a permanent view. Basically, this PR can be split to two sub-tasks:
**Task 1:** detecting a temporary view from the query plan of view definition.
When finding an unresolved temporary view, Analyzer replaces it by a `SubqueryAlias` with the corresponding logical plan, which is stored in an in-memory HashMap. After replacement, it is impossible to detect whether the `SubqueryAlias` is added/generated from a temporary view. Thus, to detect the usage of a temporary view in view definition, this PR traverses the unresolved logical plan and uses the name of an `UnresolvedRelation` to detect whether it is a (global) temporary view.
**Task 2:** detecting a temporary UDF from the query plan of view definition.
Detecting usage of a temporary UDF in view definition is not straightfoward.
First, in the analyzed plan, we are having different forms to represent the functions. More importantly, some classes (e.g., `HiveGenericUDF`) are not accessible from `CreateViewCommand`, which is part of `sql/core`. Thus, we used the unanalyzed plan `child` of `CreateViewCommand` to detect the usage of a temporary UDF. Because the plan has already been successfully analyzed, we can assume the functions have been defined/registered.
Second, in Spark, the functions have four forms: Spark built-in functions, built-in hash functions, permanent UDFs and temporary UDFs. We do not have any direct way to determine whether a function is temporary or not. Thus, we introduced a function `isTemporaryFunction` in `SessionCatalog`. This function contains the detailed logics to determine whether a function is temporary or not.
### How was this patch tested?
Added test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15764 from gatorsmile/blockTempFromPermViewCreation.
## What changes were proposed in this pull request?
This PR proposes to match up the behaviour of `to_json` to `from_json` function for null-safety.
Currently, it throws `NullPointException` but this PR fixes this to produce `null` instead.
with the data below:
```scala
import spark.implicits._
val df = Seq(Some(Tuple1(Tuple1(1))), None).toDF("a")
df.show()
```
```
+----+
| a|
+----+
| [1]|
|null|
+----+
```
the codes below
```scala
import org.apache.spark.sql.functions._
df.select(to_json($"a")).show()
```
produces..
**Before**
throws `NullPointException` as below:
```
java.lang.NullPointerException
at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeFields(JacksonGenerator.scala:138)
at org.apache.spark.sql.catalyst.json.JacksonGenerator$$anonfun$write$1.apply$mcV$sp(JacksonGenerator.scala:194)
at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeObject(JacksonGenerator.scala:131)
at org.apache.spark.sql.catalyst.json.JacksonGenerator.write(JacksonGenerator.scala:193)
at org.apache.spark.sql.catalyst.expressions.StructToJson.eval(jsonExpressions.scala:544)
at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:142)
at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:48)
at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:30)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```
**After**
```
+---------------+
|structtojson(a)|
+---------------+
| {"_1":1}|
| null|
+---------------+
```
## How was this patch tested?
Unit test in `JsonExpressionsSuite.scala` and `JsonFunctionsSuite.scala`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15792 from HyukjinKwon/SPARK-18295.
## What changes were proposed in this pull request?
Add a function to check if two integers are compatible when invoking `acceptsType()` in `DataType`.
## How was this patch tested?
Manually.
E.g.
```
spark.sql("create table t3(a map<bigint, array<string>>)")
spark.sql("select * from t3 where a[1] is not null")
```
Before:
```
cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
org.apache.spark.sql.AnalysisException: cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:82)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:74)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:307)
```
After:
Run the sql queries above. No errors.
Author: Weiqing Yang <yangweiqing001@gmail.com>
Closes#15448 from weiqingy/SPARK_17108.
## What changes were proposed in this pull request?
As reported in the jira, sometimes the generated java code in codegen will cause compilation error.
Code snippet to test it:
case class Route(src: String, dest: String, cost: Int)
case class GroupedRoutes(src: String, dest: String, routes: Seq[Route])
val ds = sc.parallelize(Array(
Route("a", "b", 1),
Route("a", "b", 2),
Route("a", "c", 2),
Route("a", "d", 10),
Route("b", "a", 1),
Route("b", "a", 5),
Route("b", "c", 6))
).toDF.as[Route]
val grped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r)))
.groupByKey(r => (r.src, r.dest))
.reduceGroups { (g1: GroupedRoutes, g2: GroupedRoutes) =>
GroupedRoutes(g1.src, g1.dest, g1.routes ++ g2.routes)
}.map(_._2)
The problem here is, in `ReferenceToExpressions` we evaluate the children vars to local variables. Then the result expression is evaluated to use those children variables. In the above case, the result expression code is too long and will be split by `CodegenContext.splitExpression`. So those local variables cannot be accessed and cause compilation error.
## How was this patch tested?
Jenkins tests.
Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#15693 from viirya/fix-codege-compilation-error.
## What changes were proposed in this pull request?
We have an undocumented naming convention to call expression unit tests ExpressionsSuite, and the end-to-end tests FunctionsSuite. It'd be great to make all test suites consistent with this naming convention.
## How was this patch tested?
This is a test-only naming change.
Author: Reynold Xin <rxin@databricks.com>
Closes#15793 from rxin/SPARK-18296.
## What changes were proposed in this pull request?
Previously `TRUNCATE TABLE ... PARTITION` will always truncate the whole table for data source tables, this PR fixes it and improve `InMemoryCatalog` to make this command work with it.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15688 from cloud-fan/truncate.
## What changes were proposed in this pull request?
This PR proposes `rand`/`randn` accept `null` as input in Scala/SQL and `LongType` as input in SQL. In this case, it treats the values as `0`.
So, this PR includes both changes below:
- `null` support
It seems MySQL also accepts this.
``` sql
mysql> select rand(0);
+---------------------+
| rand(0) |
+---------------------+
| 0.15522042769493574 |
+---------------------+
1 row in set (0.00 sec)
mysql> select rand(NULL);
+---------------------+
| rand(NULL) |
+---------------------+
| 0.15522042769493574 |
+---------------------+
1 row in set (0.00 sec)
```
and also Hive does according to [HIVE-14694](https://issues.apache.org/jira/browse/HIVE-14694)
So the codes below:
``` scala
spark.range(1).selectExpr("rand(null)").show()
```
prints..
**Before**
```
Input argument to rand must be an integer literal.;; line 1 pos 0
org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:444)
```
**After**
```
+-----------------------+
|rand(CAST(NULL AS INT))|
+-----------------------+
| 0.13385709732307427|
+-----------------------+
```
- `LongType` support in SQL.
In addition, it make the function allows to take `LongType` consistently within Scala/SQL.
In more details, the codes below:
``` scala
spark.range(1).select(rand(1), rand(1L)).show()
spark.range(1).selectExpr("rand(1)", "rand(1L)").show()
```
prints..
**Before**
```
+------------------+------------------+
| rand(1)| rand(1)|
+------------------+------------------+
|0.2630967864682161|0.2630967864682161|
+------------------+------------------+
Input argument to rand must be an integer literal.;; line 1 pos 0
org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
at
```
**After**
```
+------------------+------------------+
| rand(1)| rand(1)|
+------------------+------------------+
|0.2630967864682161|0.2630967864682161|
+------------------+------------------+
+------------------+------------------+
| rand(1)| rand(1)|
+------------------+------------------+
|0.2630967864682161|0.2630967864682161|
+------------------+------------------+
```
## How was this patch tested?
Unit tests in `DataFrameSuite.scala` and `RandomSuite.scala`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15432 from HyukjinKwon/SPARK-17854.
## What changes were proposed in this pull request?
Prior this pr, the following code would cause an NPE:
`case class point(a:String, b:String, c:String, d: Int)`
`val data = Seq(
point("1","2","3", 1),
point("4","5","6", 1),
point("7","8","9", 1)
)`
`sc.parallelize(data).toDF().registerTempTable("table")`
`spark.sql("select a, b, c, count(d) from table group by a, b, c GROUPING SETS ((a)) ").show()`
The reason is that when the grouping_id() behavior was changed in #10677, some code (which should be changed) was left out.
Take the above code for example, prior #10677, the bit mask for set "(a)" was `001`, while after #10677 the bit mask was changed to `011`. However, the `nonNullBitmask` was not changed accordingly.
This pr will fix this problem.
## How was this patch tested?
add integration tests
Author: wangyang <wangyang@haizhi.com>
Closes#15416 from yangw1234/groupingid.
## What changes were proposed in this pull request?
As the title suggests, this patch moves hash expressions from misc.scala into hash.scala, to make it easier to find the hash functions. I wanted to do this a while ago but decided to wait for the branch-2.1 cut so the chance of conflicts will be smaller.
## How was this patch tested?
Test cases were also moved out of MiscFunctionsSuite into HashExpressionsSuite.
Author: Reynold Xin <rxin@databricks.com>
Closes#15784 from rxin/SPARK-18287.
## What changes were proposed in this pull request?
For data source tables, we will put its table schema, partition columns, etc. to table properties, to work around some hive metastore issues, e.g. not case-preserving, bad decimal type support, etc.
We should also do this for hive serde tables, to reduce the difference between hive serde tables and data source tables, e.g. column names should be case preserving.
## How was this patch tested?
existing tests, and a new test in `HiveExternalCatalog`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14750 from cloud-fan/minor1.
## What changes were proposed in this pull request?
`from_json` is currently not safe against `null` rows. This PR adds a fix and a regression test for it.
## How was this patch tested?
Regression test
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#15771 from brkyvz/json_fix.
## What changes were proposed in this pull request?
The `PushDownPredicate` rule can create a wrong result if we try to push a filter containing a predicate subquery through a project when the subquery and the project share attributes (have the same source).
The current PR fixes this by making sure that we do not push down when there is a predicate subquery that outputs the same attributes as the filters new child plan.
## How was this patch tested?
Added a test to `SubquerySuite`. nsyca has done previous work this. I have taken test from his initial PR.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15761 from hvanhovell/SPARK-17337.
## What changes were proposed in this pull request?
This patch renames partitionProviderIsHive to tracksPartitionsInCatalog, as the old name was too Hive specific.
## How was this patch tested?
Should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#15750 from rxin/SPARK-18244.
## What changes were proposed in this pull request?
I was reading this part of the code and was really confused by the "partition" parameter. This patch adds some documentation for it to reduce confusion in the future.
I also looked around other logical plans but most of them are either already documented, or pretty self-evident to people that know Spark SQL.
## How was this patch tested?
N/A - doc change only.
Author: Reynold Xin <rxin@databricks.com>
Closes#15749 from rxin/doc-improvement.
## What changes were proposed in this pull request?
In Spark 1.6 and earlier, we can drop the database we are using. In Spark 2.0, native implementation prevent us from dropping current database, which may break some old queries. This PR would re-enable the feature.
## How was this patch tested?
one new unit test in `SessionCatalogSuite`.
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#15011 from adrian-wang/dropcurrent.
### What changes were proposed in this pull request?
So far, we have limited test case coverage about implicit type casting. We need to draw a matrix to find all the possible casting pairs.
- Reorged the existing test cases
- Added all the possible type casting pairs
- Drawed a matrix to show the implicit type casting. The table is very wide. Maybe hard to review. Thus, you also can access the same table via the link to [a google sheet](https://docs.google.com/spreadsheets/d/19PS4ikrs-Yye_mfu-rmIKYGnNe-NmOTt5DDT1fOD3pI/edit?usp=sharing).
SourceType\CastToType | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | BooleanType | StringType | DateType | TimestampType | ArrayType | MapType | StructType | NullType | CalendarIntervalType | DecimalType | NumericType | IntegralType
------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | -----------
**ByteType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X | X | StringType | X | X | X | X | X | X | X | DecimalType(3, 0) | ByteType | ByteType
**ShortType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X | X | StringType | X | X | X | X | X | X | X | DecimalType(5, 0) | ShortType | ShortType
**IntegerType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X | X | StringType | X | X | X | X | X | X | X | DecimalType(10, 0) | IntegerType | IntegerType
**LongType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X | X | StringType | X | X | X | X | X | X | X | DecimalType(20, 0) | LongType | LongType
**DoubleType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X | X | StringType | X | X | X | X | X | X | X | DecimalType(30, 15) | DoubleType | IntegerType
**FloatType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X | X | StringType | X | X | X | X | X | X | X | DecimalType(14, 7) | FloatType | IntegerType
**Dec(10, 2)** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X | X | StringType | X | X | X | X | X | X | X | DecimalType(10, 2) | Dec(10, 2) | IntegerType
**BinaryType** | X | X | X | X | X | X | X | BinaryType | X | StringType | X | X | X | X | X | X | X | X | X | X
**BooleanType** | X | X | X | X | X | X | X | X | BooleanType | StringType | X | X | X | X | X | X | X | X | X | X
**StringType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | X | StringType | DateType | TimestampType | X | X | X | X | X | DecimalType(38, 18) | DoubleType | X
**DateType** | X | X | X | X | X | X | X | X | X | StringType | DateType | TimestampType | X | X | X | X | X | X | X | X
**TimestampType** | X | X | X | X | X | X | X | X | X | StringType | DateType | TimestampType | X | X | X | X | X | X | X | X
**ArrayType** | X | X | X | X | X | X | X | X | X | X | X | X | ArrayType* | X | X | X | X | X | X | X
**MapType** | X | X | X | X | X | X | X | X | X | X | X | X | X | MapType* | X | X | X | X | X | X
**StructType** | X | X | X | X | X | X | X | X | X | X | X | X | X | X | StructType* | X | X | X | X | X
**NullType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | BooleanType | StringType | DateType | TimestampType | ArrayType | MapType | StructType | NullType | CalendarIntervalType | DecimalType(38, 18) | DoubleType | IntegerType
**CalendarIntervalType** | X | X | X | X | X | X | X | X | X | X | X | X | X | X | X | X | CalendarIntervalType | X | X | X
Note: ArrayType\*, MapType\*, StructType\* are castable only when the internal child types also match; otherwise, not castable
### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15691 from gatorsmile/implicitTypeCasting.
## What changes were proposed in this pull request?
This PR proposes to change the documentation for functions. Please refer the discussion from https://github.com/apache/spark/pull/15513
The changes include
- Re-indent the documentation
- Add examples/arguments in `extended` where the arguments are multiple or specific format (e.g. xml/ json).
For examples, the documentation was updated as below:
### Functions with single line usage
**Before**
- `pow`
``` sql
Usage: pow(x1, x2) - Raise x1 to the power of x2.
Extended Usage:
> SELECT pow(2, 3);
8.0
```
- `current_timestamp`
``` sql
Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
Extended Usage:
No example for current_timestamp.
```
**After**
- `pow`
``` sql
Usage: pow(expr1, expr2) - Raises `expr1` to the power of `expr2`.
Extended Usage:
Examples:
> SELECT pow(2, 3);
8.0
```
- `current_timestamp`
``` sql
Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
Extended Usage:
No example/argument for current_timestamp.
```
### Functions with (already) multiple line usage
**Before**
- `approx_count_distinct`
``` sql
Usage: approx_count_distinct(expr) - Returns the estimated cardinality by HyperLogLog++.
approx_count_distinct(expr, relativeSD=0.05) - Returns the estimated cardinality by HyperLogLog++
with relativeSD, the maximum estimation error allowed.
Extended Usage:
No example for approx_count_distinct.
```
- `percentile_approx`
``` sql
Usage:
percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
column `col` at the given percentage. The value of percentage must be between 0.0
and 1.0. The `accuracy` parameter (default: 10000) is a positive integer literal which
controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
better accuracy, `1.0/accuracy` is the relative error of the approximation.
percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy]) - Returns the approximate
percentile array of column `col` at the given percentage array. Each value of the
percentage array must be between 0.0 and 1.0. The `accuracy` parameter (default: 10000) is
a positive integer literal which controls approximation accuracy at the cost of memory.
Higher value of `accuracy` yields better accuracy, `1.0/accuracy` is the relative error of
the approximation.
Extended Usage:
No example for percentile_approx.
```
**After**
- `approx_count_distinct`
``` sql
Usage:
approx_count_distinct(expr[, relativeSD]) - Returns the estimated cardinality by HyperLogLog++.
`relativeSD` defines the maximum estimation error allowed.
Extended Usage:
No example/argument for approx_count_distinct.
```
- `percentile_approx`
``` sql
Usage:
percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
column `col` at the given percentage. The value of percentage must be between 0.0
and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
better accuracy, `1.0/accuracy` is the relative error of the approximation.
When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
In this case, returns the approximate percentile array of column `col` at the given
percentage array.
Extended Usage:
Examples:
> SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
[10.0,10.0,10.0]
> SELECT percentile_approx(10.0, 0.5, 100);
10.0
```
## How was this patch tested?
Manually tested
**When examples are multiple**
``` sql
spark-sql> describe function extended reflect;
Function: reflect
Class: org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection
Usage: reflect(class, method[, arg1[, arg2 ..]]) - Calls a method with reflection.
Extended Usage:
Examples:
> SELECT reflect('java.util.UUID', 'randomUUID');
c33fb387-8500-4bfa-81d2-6e0e3e930df2
> SELECT reflect('java.util.UUID', 'fromString', 'a5cf6c42-0c85-418f-af6c-3e4e5b1328f2');
a5cf6c42-0c85-418f-af6c-3e4e5b1328f2
```
**When `Usage` is in single line**
``` sql
spark-sql> describe function extended min;
Function: min
Class: org.apache.spark.sql.catalyst.expressions.aggregate.Min
Usage: min(expr) - Returns the minimum value of `expr`.
Extended Usage:
No example/argument for min.
```
**When `Usage` is already in multiple lines**
``` sql
spark-sql> describe function extended percentile_approx;
Function: percentile_approx
Class: org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
Usage:
percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
column `col` at the given percentage. The value of percentage must be between 0.0
and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
better accuracy, `1.0/accuracy` is the relative error of the approximation.
When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
In this case, returns the approximate percentile array of column `col` at the given
percentage array.
Extended Usage:
Examples:
> SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
[10.0,10.0,10.0]
> SELECT percentile_approx(10.0, 0.5, 100);
10.0
```
**When example/argument is missing**
``` sql
spark-sql> describe function extended rank;
Function: rank
Class: org.apache.spark.sql.catalyst.expressions.Rank
Usage:
rank() - Computes the rank of a value in a group of values. The result is one plus the number
of rows preceding or equal to the current row in the ordering of the partition. The values
will produce gaps in the sequence.
Extended Usage:
No example/argument for rank.
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15677 from HyukjinKwon/SPARK-17963-1.
## What changes were proposed in this pull request?
Due to a limitation of hive metastore(table location must be directory path, not file path), we always store `path` for data source table in storage properties, instead of the `locationUri` field. However, we should not expose this difference to `CatalogTable` level, but just treat it as a hack in `HiveExternalCatalog`, like we store table schema of data source table in table properties.
This PR unifies `path` and `locationUri` outside of `HiveExternalCatalog`, both data source table and hive serde table should use the `locationUri` field.
This PR also unifies the way we handle default table location for managed table. Previously, the default table location of hive serde managed table is set by external catalog, but the one of data source table is set by command. After this PR, we follow the hive way and the default table location is always set by external catalog.
For managed non-file-based tables, we will assign a default table location and create an empty directory for it, the table location will be removed when the table is dropped. This is reasonable as metastore doesn't care about whether a table is file-based or not, and an empty table directory has no harm.
For external non-file-based tables, ideally we can omit the table location, but due to a hive metastore issue, we will assign a random location to it, and remove it right after the table is created. See SPARK-15269 for more details. This is fine as it's well isolated in `HiveExternalCatalog`.
To keep the existing behaviour of the `path` option, in this PR we always add the `locationUri` to storage properties using key `path`, before passing storage properties to `DataSource` as data source options.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15024 from cloud-fan/path.
## What changes were proposed in this pull request?
RuntimeReplaceable is used to create aliases for expressions, but the way it deals with type coercion is pretty weird (each expression is responsible for how to handle type coercion, which does not obey the normal implicit type cast rules).
This patch simplifies its handling by allowing the analyzer to traverse into the actual expression of a RuntimeReplaceable.
## How was this patch tested?
- Correctness should be guaranteed by existing unit tests already
- Removed SQLCompatibilityFunctionSuite and moved it sql-compatibility-functions.sql
- Added a new test case in sql-compatibility-functions.sql for verifying explain behavior.
Author: Reynold Xin <rxin@databricks.com>
Closes#15723 from rxin/SPARK-18214.
## What changes were proposed in this pull request?
When a user appended a column using a "nondeterministic" function to a DataFrame, e.g., `rand`, `randn`, and `monotonically_increasing_id`, the expected semantic is the following:
- The value in each row should remain unchanged, as if we materialize the column immediately, regardless of later DataFrame operations.
However, since we use `TaskContext.getPartitionId` to get the partition index from the current thread, the values from nondeterministic columns might change if we call `union` or `coalesce` after. `TaskContext.getPartitionId` returns the partition index of the current Spark task, which might not be the corresponding partition index of the DataFrame where we defined the column.
See the unit tests below or JIRA for examples.
This PR uses the partition index from `RDD.mapPartitionWithIndex` instead of `TaskContext` and fixes the partition initialization logic in whole-stage codegen, normal codegen, and codegen fallback. `initializeStatesForPartition(partitionIndex: Int)` was added to `Projection`, `Nondeterministic`, and `Predicate` (codegen) and initialized right after object creation in `mapPartitionWithIndex`. `newPredicate` now returns a `Predicate` instance rather than a function for proper initialization.
## How was this patch tested?
Unit tests. (Actually I'm not very confident that this PR fixed all issues without introducing new ones ...)
cc: rxin davies
Author: Xiangrui Meng <meng@databricks.com>
Closes#15567 from mengxr/SPARK-14393.
## What changes were proposed in this pull request?
This pr is to add pattern-matching entries for array data in `Literal.apply`.
## How was this patch tested?
Added tests in `LiteralExpressionSuite`.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#15257 from maropu/SPARK-17683.
## What changes were proposed in this pull request?
Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.
This PR includes:
1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.
## How was this patch tested?
Running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.
Modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.
Author: eyal farago <eyal farago>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: eyal farago <eyal.farago@gmail.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>
Closes#15718 from hvanhovell/SPARK-16839-2.
## What changes were proposed in this pull request?
Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat`
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#15610 from srowen/SPARK-18076.
## What changes were proposed in this pull request?
There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive.
(1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition.
(2) INSERT|OVERWRITE does not work with partitions that have custom locations.
This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when `manageFilesourcePartitions = false` is unchanged.
There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release.
## How was this patch tested?
Unit tests.
Author: Eric Liang <ekl@databricks.com>
Closes#15705 from ericl/sc-4942.
## What changes were proposed in this pull request?
This PR proposes to add `to_json` function in contrast with `from_json` in Scala, Java and Python.
It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function.
The usage is as below:
``` scala
val df = Seq(Tuple1(Tuple1(1))).toDF("a")
df.select(to_json($"a").as("json")).show()
```
``` bash
+--------+
| json|
+--------+
|{"_1":1}|
+--------+
```
## How was this patch tested?
Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15354 from HyukjinKwon/SPARK-17764.
## What changes were proposed in this pull request?
Aggregation Without Window/GroupBy expressions will fail in `checkAnalysis`, the error message is a bit misleading, we should generate a more specific error message for this case.
For example,
```
spark.read.load("/some-data")
.withColumn("date_dt", to_date($"date"))
.withColumn("year", year($"date_dt"))
.withColumn("week", weekofyear($"date_dt"))
.withColumn("user_count", count($"userId"))
.withColumn("daily_max_in_week", max($"user_count").over(weeklyWindow))
)
```
creates the following output:
```
org.apache.spark.sql.AnalysisException: expression '`randomColumn`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;
```
In the error message above, `randomColumn` doesn't appear in the query(acturally it's added by function `withColumn`), so the message is not enough for the user to address the problem.
## How was this patch tested?
Manually test
Before:
```
scala> spark.sql("select col, count(col) from tbl")
org.apache.spark.sql.AnalysisException: expression 'tbl.`col`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
```
After:
```
scala> spark.sql("select col, count(col) from tbl")
org.apache.spark.sql.AnalysisException: grouping expressions sequence is empty, and 'tbl.`col`' is not an aggregate function. Wrap '(count(col#231L) AS count(col)#239L)' in windowing function(s) or wrap 'tbl.`col`' in first() (or first_value) if you don't care which value you get.;;
```
Also add new test sqls in `group-by.sql`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15672 from jiangxb1987/groupBy-empty.
## What changes were proposed in this pull request?
Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.
This PR includes:
1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.
## How was this patch tested?
running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.
modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.
Credit goes to hvanhovell for assisting with this PR.
Author: eyal farago <eyal farago>
Author: eyal farago <eyal.farago@gmail.com>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>
Closes#14444 from eyalfa/SPARK-16839_redundant_aliases_after_cleanupAliases.
## What changes were proposed in this pull request?
Currently an unqualified `getFunction(..)`call returns a wrong result; the returned function is shown as temporary function without a database. For example:
```
scala> sql("create function fn1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs'")
res0: org.apache.spark.sql.DataFrame = []
scala> spark.catalog.getFunction("fn1")
res1: org.apache.spark.sql.catalog.Function = Function[name='fn1', className='org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs', isTemporary='true']
```
This PR fixes this by adding database information to ExpressionInfo (which is used to store the function information).
## How was this patch tested?
Added more thorough tests to `CatalogSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15542 from hvanhovell/SPARK-17996.
## What changes were proposed in this pull request?
When multiple records have the minimum value, the answer of ApproximatePercentile is wrong.
## How was this patch tested?
add a test case
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#15641 from wzhfy/percentile.
## What changes were proposed in this pull request?
We should follow hive table and also store partition spec in metastore for data source table.
This brings 2 benefits:
1. It's more flexible to manage the table data files, as users can use `ADD PARTITION`, `DROP PARTITION` and `RENAME PARTITION`
2. We don't need to cache all file status for data source table anymore.
## How was this patch tested?
existing tests.
Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15515 from cloud-fan/partition.
## What changes were proposed in this pull request?
In order to facilitate the writing of additional Encoders, I proposed opening up the ObjectType SQL DataType. This DataType is used extensively in the JavaBean Encoder, but would also be useful in writing other custom encoders.
As mentioned by marmbrus, it is understood that the Expressions API is subject to potential change.
## How was this patch tested?
The change only affects the visibility of the ObjectType class, and the existing SQL test suite still runs without error.
Author: ALeksander Eskilson <alek.eskilson@cerner.com>
Closes#15453 from bdrillard/master.
## What changes were proposed in this pull request?
The `UnaryNode.getAliasedConstraints` function fails to replace all expressions by their alias where constraints contains more than one expression to be replaced.
For example:
```
val tr = LocalRelation('a.int, 'b.string, 'c.int)
val multiAlias = tr.where('a === 'c + 10).select('a.as('x), 'c.as('y))
multiAlias.analyze.constraints
```
currently outputs:
```
ExpressionSet(Seq(
IsNotNull(resolveColumn(multiAlias.analyze, "x")),
IsNotNull(resolveColumn(multiAlias.analyze, "y"))
)
```
The constraint `resolveColumn(multiAlias.analyze, "x") === resolveColumn(multiAlias.analyze, "y") + 10)` is missing.
## How was this patch tested?
Add new test cases in `ConstraintPropagationSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15597 from jiangxb1987/alias-constraints.
## What changes were proposed in this pull request?
The function `QueryPlan.inferAdditionalConstraints` and `UnaryNode.getAliasedConstraints` can produce a non-converging set of constraints for recursive functions. For instance, if we have two constraints of the form(where a is an alias):
`a = b, a = f(b, c)`
Applying both these rules in the next iteration would infer:
`f(b, c) = f(f(b, c), c)`
This process repeated, the iteration won't converge and the set of constraints will grow larger and larger until OOM.
~~To fix this problem, we collect alias from expressions and skip infer constraints if we are to transform an `Expression` to another which contains it.~~
To fix this problem, we apply additional check in `inferAdditionalConstraints`, when it's possible to generate recursive constraints, we skip generate that.
## How was this patch tested?
Add new testcase in `SQLQuerySuite`/`InferFiltersFromConstraintsSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15319 from jiangxb1987/constraints.
## What changes were proposed in this pull request?
Binary operator requires its inputs to be of same type, but it should not consider nullability, e.g. `EqualTo` should be able to compare an element-nullable array and an element-non-nullable array.
## How was this patch tested?
a regression test in `DataFrameSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15606 from cloud-fan/type-bug.
## What changes were proposed in this pull request?
Currently we always lowercase the partition columns of partition spec in parser, with the assumption that table partition columns are always lowercased.
However, this is not true for data source tables, which are case preserving. It's safe for now because data source tables don't store partition spec in metastore and don't support `ADD PARTITION`, `DROP PARTITION`, `RENAME PARTITION`, but we should make our code future-proof.
This PR makes partition spec case preserving at parser, and improve the `PreprocessTableInsertion` analyzer rule to normalize the partition columns in partition spec, w.r.t. the table partition columns.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15566 from cloud-fan/partition-spec.
## What changes were proposed in this pull request?
Simplify/cleanup TableFileCatalog:
1. pass a `CatalogTable` instead of `databaseName` and `tableName` into `TableFileCatalog`, so that we don't need to fetch table metadata from metastore again
2. In `TableFileCatalog.filterPartitions0`, DO NOT set `PartitioningAwareFileCatalog.BASE_PATH_PARAM`. According to the [classdoc](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L189-L209), the default value of `basePath` already satisfies our need. What's more, if we set this parameter, we may break the case 2 which is metioned in the classdoc.
3. add `equals` and `hashCode` to `TableFileCatalog`
4. add `SessionCatalog.listPartitionsByFilter` which handles case sensitivity.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15568 from cloud-fan/table-file-catalog.
## What changes were proposed in this pull request?
The PR tries to fix [SPARK-18058](https://issues.apache.org/jira/browse/SPARK-18058) which refers to a bug that the column types are compared with the extra care about Nullability in Union and SetOperation.
This PR converts the columns types by setting all fields as nullable before comparison
## How was this patch tested?
regular unit test cases
Author: CodingCat <zhunansjtu@gmail.com>
Closes#15595 from CodingCat/SPARK-18058.
## What changes were proposed in this pull request?
Jira: https://issues.apache.org/jira/browse/SPARK-18035
In HiveInspectors, I saw that converting Java map to Spark's `ArrayBasedMapData` spent quite sometime in buffer copying : https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala#L658
The reason being `map.toSeq` allocates a new buffer and copies the map entries to it: https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/MapLike.scala#L323
This copy is not needed as we get rid of it once we extract the key and value arrays.
Here is the call trace:
```
org.apache.spark.sql.hive.HiveInspectors$$anonfun$unwrapperFor$41.apply(HiveInspectors.scala:664)
scala.collection.AbstractMap.toSeq(Map.scala:59)
scala.collection.MapLike$class.toSeq(MapLike.scala:323)
scala.collection.AbstractMap.toBuffer(Map.scala:59)
scala.collection.MapLike$class.toBuffer(MapLike.scala:326)
scala.collection.AbstractTraversable.copyToBuffer(Traversable.scala:104)
scala.collection.TraversableOnce$class.copyToBuffer(TraversableOnce.scala:275)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
scala.collection.AbstractIterable.foreach(Iterable.scala:54)
scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
scala.collection.Iterator$class.foreach(Iterator.scala:893)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
```
Also, earlier code was populating keys and values arrays separately by iterating twice. The PR avoids double iteration of the map and does it in one iteration.
EDIT: During code review, there were several more places in the code which were found to do similar thing. The PR dedupes those instances and introduces convenient APIs which are performant and memory efficient
## Performance gains
The number is subjective and depends on how many map columns are accessed in the query and average entries per map. For one the queries that I tried out, I saw 3% CPU savings (end-to-end) for the query.
## How was this patch tested?
This does not change the end result produced so relying on existing tests.
Author: Tejas Patil <tejasp@fb.com>
Closes#15573 from tejasapatil/SPARK-18035_avoid_toSeq.
## What changes were proposed in this pull request?
`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.
cc srowen
## How was this patch tested?
existing tests
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Closes#15564 from zhengruifeng/avoid_0_length_array.
## What changes were proposed in this pull request?
In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it.
## How was this patch tested?
the new `PruneFileSourcePartitionsSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15569 from cloud-fan/partition-bug.
## What changes were proposed in this pull request?
Add mapValues to KeyValueGroupedDataset
## How was this patch tested?
New test in DatasetSuite for groupBy function, mapValues, flatMap
Author: Koert Kuipers <koert@tresata.com>
Closes#13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues.
## What changes were proposed in this pull request?
Jira : https://issues.apache.org/jira/browse/SPARK-17698
`ExtractEquiJoinKeys` is incorrectly using filter predicates as the join condition for joins. `canEvaluate` [0] tries to see if the an `Expression` can be evaluated using output of a given `Plan`. In case of filter predicates (eg. `a.id='1'`), the `Expression` passed for the right hand side (ie. '1' ) is a `Literal` which does not have any attribute references. Thus `expr.references` is an empty set which theoretically is a subset of any set. This leads to `canEvaluate` returning `true` and `a.id='1'` is treated as a join predicate. While this does not lead to incorrect results but in case of bucketed + sorted tables, we might miss out on avoiding un-necessary shuffle + sort. See example below:
[0] : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L91
eg.
```
val df = (1 until 10).toDF("id").coalesce(1)
hc.sql("DROP TABLE IF EXISTS table1").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table1")
hc.sql("DROP TABLE IF EXISTS table2").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table2")
sqlContext.sql("""
SELECT a.id, b.id
FROM table1 a
FULL OUTER JOIN table2 b
ON a.id = b.id AND a.id='1' AND b.id='1'
""").explain(true)
```
BEFORE: This is doing shuffle + sort over table scan outputs which is not needed as both tables are bucketed and sorted on the same columns and have same number of buckets. This should be a single stage job.
```
SortMergeJoin [id#38, cast(id#38 as double), 1.0], [id#39, 1.0, cast(id#39 as double)], FullOuter
:- *Sort [id#38 ASC NULLS FIRST, cast(id#38 as double) ASC NULLS FIRST, 1.0 ASC NULLS FIRST], false, 0
: +- Exchange hashpartitioning(id#38, cast(id#38 as double), 1.0, 200)
: +- *FileScan parquet default.table1[id#38] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *Sort [id#39 ASC NULLS FIRST, 1.0 ASC NULLS FIRST, cast(id#39 as double) ASC NULLS FIRST], false, 0
+- Exchange hashpartitioning(id#39, 1.0, cast(id#39 as double), 200)
+- *FileScan parquet default.table2[id#39] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```
AFTER :
```
SortMergeJoin [id#32], [id#33], FullOuter, ((cast(id#32 as double) = 1.0) && (cast(id#33 as double) = 1.0))
:- *FileScan parquet default.table1[id#32] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *FileScan parquet default.table2[id#33] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```
## How was this patch tested?
- Added a new test case for this scenario : `SPARK-17698 Join predicates should not contain filter clauses`
- Ran all the tests in `BucketedReadSuite`
Author: Tejas Patil <tejasp@fb.com>
Closes#15272 from tejasapatil/SPARK-17698_join_predicate_filter_clause.
## What changes were proposed in this pull request?
This PR proposes to check the second argument, `ascendingOrder` rather than throwing `ClassCastException` exception message.
```sql
select sort_array(array('b', 'd'), '1');
```
**Before**
```
16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')]
java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean
at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85)
at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185)
at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416)
at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50)
at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297)
```
**After**
```
Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7;
```
## How was this patch tested?
Unit test in `DataFrameFunctionsSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15532 from HyukjinKwon/SPARK-17989.
## What changes were proposed in this pull request?
Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`.
However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO.
This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc.
## How was this patch tested?
The added back tests and some new tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15434 from cloud-fan/revert.
### What changes were proposed in this pull request?
Dataset always does eager analysis now. Thus, `spark.sql.eagerAnalysis` is not used any more. Thus, we need to remove it.
This PR also outputs the plan. Without the fix, the analysis error is like
```
cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12
```
After the fix, the analysis error becomes:
```
org.apache.spark.sql.AnalysisException: cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12;
'Project [unresolvedalias(CASE WHEN ('k1 = 2) THEN 22 WHEN ('k1 = 4) THEN 44 ELSE 0 END, None), v#6]
+- SubqueryAlias t
+- Project [_1#2 AS k#5, _2#3 AS v#6]
+- LocalRelation [_1#2, _2#3]
```
### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15316 from gatorsmile/eagerAnalysis.
## What changes were proposed in this pull request?
Added a `prettyname` for current_database function.
## How was this patch tested?
Manually.
Before:
```
scala> sql("select current_database()").show
+-----------------+
|currentdatabase()|
+-----------------+
| default|
+-----------------+
```
After:
```
scala> sql("select current_database()").show
+------------------+
|current_database()|
+------------------+
| default|
+------------------+
```
Author: Weiqing Yang <yangweiqing001@gmail.com>
Closes#15506 from weiqingy/prettyName.
(This PR addresses https://issues.apache.org/jira/browse/SPARK-16980.)
## What changes were proposed in this pull request?
In a new Spark session, when a partitioned Hive table is converted to use Spark's `HadoopFsRelation` in `HiveMetastoreCatalog`, metadata for every partition of that table are retrieved from the metastore and loaded into driver memory. In addition, every partition's metadata files are read from the filesystem to perform schema inference.
If a user queries such a table with predicates which prune that table's partitions, we would like to be able to answer that query without consulting partition metadata which are not involved in the query. When querying a table with a large number of partitions for some data from a small number of partitions (maybe even a single partition), the current conversion strategy is highly inefficient. I suspect this scenario is not uncommon in the wild.
In addition to being inefficient in running time, the current strategy is inefficient in its use of driver memory. When the sum of the number of partitions of all tables loaded in a driver reaches a certain level (somewhere in the tens of thousands), their cached data exhaust all driver heap memory in the default configuration. I suspect this scenario is less common (in that not too many deployments work with tables with tens of thousands of partitions), however this does illustrate how large the memory footprint of this metadata can be. With tables with hundreds or thousands of partitions, I would expect the `HiveMetastoreCatalog` table cache to represent a significant portion of the driver's heap space.
This PR proposes an alternative approach. Basically, it makes four changes:
1. It adds a new method, `listPartitionsByFilter` to the Catalyst `ExternalCatalog` trait which returns the partition metadata for a given sequence of partition pruning predicates.
1. It refactors the `FileCatalog` type hierarchy to include a new `TableFileCatalog` to efficiently return files only for partitions matching a sequence of partition pruning predicates.
1. It removes partition loading and caching from `HiveMetastoreCatalog`.
1. It adds a new Catalyst optimizer rule, `PruneFileSourcePartitions`, which applies a plan's partition-pruning predicates to prune out unnecessary partition files from a `HadoopFsRelation`'s underlying file catalog.
The net effect is that when a query over a partitioned Hive table is planned, the analyzer retrieves the table metadata from `HiveMetastoreCatalog`. As part of this operation, the `HiveMetastoreCatalog` builds a `HadoopFsRelation` with a `TableFileCatalog`. It does not load any partition metadata or scan any files. The optimizer prunes-away unnecessary table partitions by sending the partition-pruning predicates to the relation's `TableFileCatalog `. The `TableFileCatalog` in turn calls the `listPartitionsByFilter` method on its external catalog. This queries the Hive metastore, passing along those filters.
As a bonus, performing partition pruning during optimization leads to a more accurate relation size estimate. This, along with c481bdf, can lead to automatic, safe application of the broadcast optimization in a join where it might previously have been omitted.
## Open Issues
1. This PR omits partition metadata caching. I can add this once the overall strategy for the cold path is established, perhaps in a future PR.
1. This PR removes and omits partitioned Hive table schema reconciliation. As a result, it fails to find Parquet schema columns with upper case letters because of the Hive metastore's case-insensitivity. This issue may be fixed by #14750, but that PR appears to have stalled. ericl has contributed to this PR a workaround for Parquet wherein schema reconciliation occurs at query execution time instead of planning. Whether ORC requires a similar patch is an open issue.
1. This PR omits an implementation of `listPartitionsByFilter` for the `InMemoryCatalog`.
1. This PR breaks parquet log output redirection during query execution. I can work around this by running `Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$")` first thing in a Spark shell session, but I haven't figured out how to fix this properly.
## How was this patch tested?
The current Spark unit tests were run, and some ad-hoc tests were performed to validate that only the necessary partition metadata is loaded.
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>
Closes#14690 from mallman/spark-16980-lazy_partition_fetching.
Currently pyspark can only call the builtin java UDF, but can not call custom java UDF. It would be better to allow that. 2 benefits:
* Leverage the power of rich third party java library
* Improve the performance. Because if we use python UDF, python daemons will be started on worker which will affect the performance.
Author: Jeff Zhang <zjffdu@apache.org>
Closes#9766 from zjffdu/SPARK-11775.
## What changes were proposed in this pull request?
We are trying to resolve the attribute in sort by pulling up some column for grandchild into child, but that's wrong when the child is Distinct, because the added column will change the behavior of Distinct, we should not do that.
## How was this patch tested?
Added regression test.
Author: Davies Liu <davies@databricks.com>
Closes#15489 from davies/order_distinct.
## What changes were proposed in this pull request?
`HiveExternalCatalog` should be the only interface to talk to the hive metastore. In `MetastoreRelation` we can just use `ExternalCatalog` instead of `HiveClient` to interact with hive metastore, and add missing API in `ExternalCatalog`.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15460 from cloud-fan/relation.
## What changes were proposed in this pull request?
Value classes were unsupported because catalyst data types were
obtained through reflection on erased types, which would resolve to a
value class' wrapped type and hence lead to unavailable methods during
code generation.
E.g. the following class
```scala
case class Foo(x: Int) extends AnyVal
```
would be seen as an `int` in catalyst and will cause instance cast failures when generated java code tries to treat it as a `Foo`.
This patch simply removes the erasure step when getting data types for
catalyst.
## How was this patch tested?
Additional tests in `ExpressionEncoderSuite`.
Author: Jakob Odersky <jakob@odersky.com>
Closes#15284 from jodersky/value-classes.
## What changes were proposed in this pull request?
Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics.
https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing
Specifically, this PR adds the following public APIs changes.
### New APIs
- `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later)
- `StreamingQueryStatus` has the following important fields
- inputRate - Current rate (rows/sec) at which data is being generated by all the sources
- processingRate - Current rate (rows/sec) at which the query is processing data from
all the sources
- ~~outputRate~~ - *Does not work with wholestage codegen*
- latency - Current average latency between the data being available in source and the sink writing the corresponding output
- sourceStatuses: Array[SourceStatus] - Current statuses of the sources
- sinkStatus: SinkStatus - Current status of the sink
- triggerStatus - Low-level detailed status of the last completed/currently active trigger
- latencies - getOffset, getBatch, full trigger, wal writes
- timestamps - trigger start, finish, after getOffset, after getBatch
- numRows - input, output, state total/updated rows for aggregations
- `SourceStatus` has the following important fields
- inputRate - Current rate (rows/sec) at which data is being generated by the source
- processingRate - Current rate (rows/sec) at which the query is processing data from the source
- triggerStatus - Low-level detailed status of the last completed/currently active trigger
- Python API for `StreamingQuery.status()`
### Breaking changes to existing APIs
**Existing direct public facing APIs**
- Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`.
- Branch 2.0 should have it deprecated, master should have it removed.
**Existing advanced listener APIs**
- `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus`
- Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status)
- Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`.
- Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`.
- For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java.
## How was this patch tested?
Old and new unit tests.
- Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite.
- New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite.
- New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite.
- Source-specific tests for making sure input rows are counted are is source-specific test suites.
- Additional tests to test minor additions in LocalTableScanExec, StateStore, etc.
Metrics also manually tested using Ganglia sink
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#15307 from tdas/SPARK-17731.
## What changes were proposed in this pull request?
correct the expected type from Length function to be Int
## How was this patch tested?
Test runs on little endian and big endian platforms
Author: Pete Robbins <robbinspg@gmail.com>
Closes#15464 from robbinspg/SPARK-17827.
## What changes were proposed in this pull request?
minor doc fix for "getAnyValAs" in class Row
## How was this patch tested?
None.
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: buzhihuojie <ren.weiluo@gmail.com>
Closes#15452 from david-weiluo-ren/minorDocFixForRow.
## What changes were proposed in this pull request?
This change adds a check in castToInterval method of Cast expression , such that if converted value is null , then isNull variable should be set to true.
Earlier, the expression Cast(Literal(), CalendarIntervalType) was throwing NullPointerException because of the above mentioned reason.
## How was this patch tested?
Added test case in CastSuite.scala
jira entry for detail: https://issues.apache.org/jira/browse/SPARK-17884
Author: prigarg <prigarg@adobe.com>
Closes#15449 from priyankagargnitk/SPARK-17884.
## What changes were proposed in this pull request?
SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897.
Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf.
## How was this patch tested?
new tests in SQLConfSuite
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15295 from cloud-fan/global-conf.
## What changes were proposed in this pull request?
Currently `Canonicalize` object doesn't support `And` and `Or`. So we can compare canonicalized form of predicates consistently. We should add the support.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#15388 from viirya/canonicalize-and-or.
## What changes were proposed in this pull request?
The data type API has not been changed since Spark 1.3.0, and is ready for graduation. This patch marks them as stable APIs using the new InterfaceStability annotation.
This patch also looks at the various files in the catalyst module (not the "package") and marks the remaining few classes appropriately as well.
## How was this patch tested?
This is an annotation change. No functional changes.
Author: Reynold Xin <rxin@databricks.com>
Closes#15426 from rxin/SPARK-17864.
## What changes were proposed in this pull request?
address post hoc review comments for https://github.com/apache/spark/pull/14897
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15424 from cloud-fan/global-temp-view.
## What changes were proposed in this pull request?
Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.
changes for `SessionCatalog`:
1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.
changes for SQL commands:
1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.
changes for other public API
1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`
## How was this patch tested?
new tests in `SQLViewSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14897 from cloud-fan/global-temp-view.
## What changes were proposed in this pull request?
Currently we use the same rule to parse top level and nested data fields. For example:
```
create table tbl_x(
id bigint,
nested struct<col1:string,col2:string>
)
```
Shows both syntaxes. In this PR we split this rule in a top-level and nested rule.
Before this PR,
```
sql("CREATE TABLE my_tab(column1: INT)")
```
works fine.
After this PR, it will throw a `ParseException`:
```
scala> sql("CREATE TABLE my_tab(column1: INT)")
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27)
```
## How was this patch tested?
Add new testcases in `SparkSqlParserSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15346 from jiangxb1987/cdt.
## What changes were proposed in this pull request?
The `quotedString` method in `TableIdentifier` and `FunctionIdentifier` produce an illegal (un-parseable) name when the name contains a backtick. For example:
```
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1"))
parseTableIdentifier(complexName.unquotedString) // Does not work
parseTableIdentifier(complexName.quotedString) // Does not work
parseExpression(complexName.unquotedString) // Does not work
parseExpression(complexName.quotedString) // Does not work
```
We should handle the backtick properly to make `quotedString` parseable.
## How was this patch tested?
Add new testcases in `TableIdentifierParserSuite` and `ExpressionParserSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15403 from jiangxb1987/backtick.
## What changes were proposed in this pull request?
In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`.
The code below illustrates the immutability issue with InternalRow:
```scala
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
val struct = new GenericMutableRow(1)
val row = InternalRow(struct, 1)
println(row)
scala> [[null], 1]
struct.setInt(0, 42)
println(row)
scala> [[42], 1]
```
This might be somewhat controversial, so feedback is appreciated.
## How was this patch tested?
Existing tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15333 from hvanhovell/SPARK-17761.
## What changes were proposed in this pull request?
Currently, Spark raises `RuntimeException` when creating a view with timestamp with INTERVAL arithmetic like the following. The root cause is the arithmetic expression, `TimeAdd`, was transformed into `timeadd` function as a VIEW definition. This PR fixes the SQL definition of `TimeAdd` and `TimeSub` expressions.
```scala
scala> sql("CREATE TABLE dates (ts TIMESTAMP)")
scala> sql("CREATE VIEW view1 AS SELECT ts + INTERVAL 1 DAY FROM dates")
java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ...
```
## How was this patch tested?
Pass Jenkins with a new testcase.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#15318 from dongjoon-hyun/SPARK-17750.
## What changes were proposed in this pull request?
The result of the `Last` function can be wrong when the last partition processed is empty. It can return `null` instead of the expected value. For example, this can happen when we process partitions in the following order:
```
- Partition 1 [Row1, Row2]
- Partition 2 [Row3]
- Partition 3 []
```
In this case the `Last` function will currently return a null, instead of the value of `Row3`.
This PR fixes this by adding a `valueSet` flag to the `Last` function.
## How was this patch tested?
We only used end to end tests for `DeclarativeAggregateFunction`s. I have added an evaluator for these functions so we can tests them in catalyst. I have added a `LastTestSuite` to test the `Last` aggregate function.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15348 from hvanhovell/SPARK-17758.
## What changes were proposed in this pull request?
This PR fixes the following NPE scenario in two ways.
**Reported Error Scenario**
```scala
scala> sql("EXPLAIN DESCRIBE TABLE x").show(truncate = false)
INFO SparkSqlParser: Parsing command: EXPLAIN DESCRIBE TABLE x
java.lang.NullPointerException
```
- **DESCRIBE**: Extend `DESCRIBE` syntax to accept `TABLE`.
- **EXPLAIN**: Prevent NPE in case of the parsing failure of target statement, e.g., `EXPLAIN DESCRIBE TABLES x`.
## How was this patch tested?
Pass the Jenkins test with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#15357 from dongjoon-hyun/SPARK-17328.
## What changes were proposed in this pull request?
Currently Spark SQL parses regular decimal literals (e.g. `10.00`) as decimals and scientific decimal literals (e.g. `10.0e10`) as doubles. The difference between the two confuses most users. This PR unifies the parsing behavior and also parses scientific decimal literals as decimals.
This implications in tests are limited to a single Hive compatibility test.
## How was this patch tested?
Updated tests in `ExpressionParserSuite` and `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14828 from hvanhovell/SPARK-17258.
## What changes were proposed in this pull request?
Code generation including too many mutable states exceeds JVM size limit to extract values from `references` into fields in the constructor.
We should split the generated extractions in the constructor into smaller functions.
## How was this patch tested?
I added some tests to check if the generated codes for the expressions exceed or not.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#15275 from ueshin/issues/SPARK-17702.
## What changes were proposed in this pull request?
We currently only allow relatively simple expressions as the input for a value based case statement. Expressions like `case (a > 1) or (b = 2) when true then 1 when false then 0 end` currently fail. This PR adds support for such expressions.
## How was this patch tested?
Added a test to the ExpressionParserSuite.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15322 from hvanhovell/SPARK-17753.
## What changes were proposed in this pull request?
Generate basic column statistics for all the atomic types:
- numeric types: max, min, num of nulls, ndv (number of distinct values)
- date/timestamp types: they are also represented as numbers internally, so they have the same stats as above.
- string: avg length, max length, num of nulls, ndv
- binary: avg length, max length, num of nulls
- boolean: num of nulls, num of trues, num of falsies
Also support storing and loading these statistics.
One thing to notice:
We support analyzing columns independently, e.g.:
sql1: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key;`
sql2: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS value;`
when running sql2 to collect column stats for `value`, we don’t remove stats of columns `key` which are analyzed in sql1 and not in sql2. As a result, **users need to guarantee consistency** between sql1 and sql2. If the table has been changed before sql2, users should re-analyze column `key` when they want to analyze column `value`:
`ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key, value;`
## How was this patch tested?
add unit tests
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#15090 from wzhfy/colStats.
## What changes were proposed in this pull request?
Currently, Spark does not collapse adjacent windows with the same partitioning and sorting. This PR implements `CollapseWindow` optimizer to do the followings.
1. If the partition specs and order specs are the same, collapse into the parent.
2. If the partition specs are the same and one order spec is a prefix of the other, collapse to the more specific one.
For example:
```scala
val df = spark.range(1000).select($"id" % 100 as "grp", $"id", rand() as "col1", rand() as "col2")
// Add summary statistics for all columns
import org.apache.spark.sql.expressions.Window
val cols = Seq("id", "col1", "col2")
val window = Window.partitionBy($"grp").orderBy($"id")
val result = cols.foldLeft(df) { (base, name) =>
base.withColumn(s"${name}_avg", avg(col(name)).over(window))
.withColumn(s"${name}_stddev", stddev(col(name)).over(window))
.withColumn(s"${name}_min", min(col(name)).over(window))
.withColumn(s"${name}_max", max(col(name)).over(window))
}
```
**Before**
```scala
scala> result.explain
== Physical Plan ==
Window [max(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_max#234], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [min(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_min#216], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [stddev_samp(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_stddev#191], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [avg(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_avg#167], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [max(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_max#152], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [min(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_min#138], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [stddev_samp(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_stddev#117], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [avg(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_avg#97], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [max(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_max#86L], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [min(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_min#76L], [grp#17L], [id#14L ASC NULLS FIRST]
+- *Project [grp#17L, id#14L, col1#18, col2#19, id_avg#26, id_stddev#42]
+- Window [stddev_samp(_w0#59) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_stddev#42], [grp#17L], [id#14L ASC NULLS FIRST]
+- *Project [grp#17L, id#14L, col1#18, col2#19, id_avg#26, cast(id#14L as double) AS _w0#59]
+- Window [avg(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_avg#26], [grp#17L], [id#14L ASC NULLS FIRST]
+- *Sort [grp#17L ASC NULLS FIRST, id#14L ASC NULLS FIRST], false, 0
+- Exchange hashpartitioning(grp#17L, 200)
+- *Project [(id#14L % 100) AS grp#17L, id#14L, rand(-6329949029880411066) AS col1#18, rand(-7251358484380073081) AS col2#19]
+- *Range (0, 1000, step=1, splits=Some(8))
```
**After**
```scala
scala> result.explain
== Physical Plan ==
Window [max(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_max#220, min(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_min#202, stddev_samp(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_stddev#177, avg(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_avg#153, max(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_max#138, min(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_min#124, stddev_samp(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_stddev#103, avg(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_avg#83, max(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_max#72L, min(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_min#62L], [grp#3L], [id#0L ASC NULLS FIRST]
+- *Project [grp#3L, id#0L, col1#4, col2#5, id_avg#12, id_stddev#28]
+- Window [stddev_samp(_w0#45) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_stddev#28], [grp#3L], [id#0L ASC NULLS FIRST]
+- *Project [grp#3L, id#0L, col1#4, col2#5, id_avg#12, cast(id#0L as double) AS _w0#45]
+- Window [avg(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_avg#12], [grp#3L], [id#0L ASC NULLS FIRST]
+- *Sort [grp#3L ASC NULLS FIRST, id#0L ASC NULLS FIRST], false, 0
+- Exchange hashpartitioning(grp#3L, 200)
+- *Project [(id#0L % 100) AS grp#3L, id#0L, rand(6537478539664068821) AS col1#4, rand(-8961093871295252795) AS col2#5]
+- *Range (0, 1000, step=1, splits=Some(8))
```
## How was this patch tested?
Pass the Jenkins tests with a newly added testsuite.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#15317 from dongjoon-hyun/SPARK-17739.
## What changes were proposed in this pull request?
There are many minor objects in references, which are extracted to the generated class field, e.g. `errMsg` in `GetExternalRowField` or `ValidateExternalType`, but number of fields in class is limited so we should reduce the number.
This pr adds unnamed version of `addReferenceObj` for these minor objects not to store the object into field but refer it from the `references` field at the time of use.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#15276 from ueshin/issues/SPARK-17703.
## What changes were proposed in this pull request?
Currently for `Union [Distinct]`, a `Distinct` operator is necessary to be on the top of `Union`. Once there are adjacent `Union [Distinct]`, there will be multiple `Distinct` in the query plan.
E.g.,
For a query like: select 1 a union select 2 b union select 3 c
Before this patch, its physical plan looks like:
*HashAggregate(keys=[a#13], functions=[])
+- Exchange hashpartitioning(a#13, 200)
+- *HashAggregate(keys=[a#13], functions=[])
+- Union
:- *HashAggregate(keys=[a#13], functions=[])
: +- Exchange hashpartitioning(a#13, 200)
: +- *HashAggregate(keys=[a#13], functions=[])
: +- Union
: :- *Project [1 AS a#13]
: : +- Scan OneRowRelation[]
: +- *Project [2 AS b#14]
: +- Scan OneRowRelation[]
+- *Project [3 AS c#15]
+- Scan OneRowRelation[]
Only the top distinct should be necessary.
After this patch, the physical plan looks like:
*HashAggregate(keys=[a#221], functions=[], output=[a#221])
+- Exchange hashpartitioning(a#221, 5)
+- *HashAggregate(keys=[a#221], functions=[], output=[a#221])
+- Union
:- *Project [1 AS a#221]
: +- Scan OneRowRelation[]
:- *Project [2 AS b#222]
: +- Scan OneRowRelation[]
+- *Project [3 AS c#223]
+- Scan OneRowRelation[]
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#15238 from viirya/remove-extra-distinct-union.
Spark SQL has great support for reading text files that contain JSON data. However, in many cases the JSON data is just one column amongst others. This is particularly true when reading from sources such as Kafka. This PR adds a new functions `from_json` that converts a string column into a nested `StructType` with a user specified schema.
Example usage:
```scala
val df = Seq("""{"a": 1}""").toDS()
val schema = new StructType().add("a", IntegerType)
df.select(from_json($"value", schema) as 'json) // => [json: <a: int>]
```
This PR adds support for java, scala and python. I leveraged our existing JSON parsing support by moving it into catalyst (so that we could define expressions using it). I left SQL out for now, because I'm not sure how users would specify a schema.
Author: Michael Armbrust <michael@databricks.com>
Closes#15274 from marmbrus/jsonParser.
## What changes were proposed in this pull request?
This patch fixes a minor correctness issue impacting the pushdown of filters beneath aggregates. Specifically, if a filter condition references no grouping or aggregate columns (e.g. `WHERE false`) then it would be incorrectly pushed beneath an aggregate.
Intuitively, the only case where you can push a filter beneath an aggregate is when that filter is deterministic and is defined over the grouping columns / expressions, since in that case the filter is acting to exclude entire groups from the query (like a `HAVING` clause). The existing code would only push deterministic filters beneath aggregates when all of the filter's references were grouping columns, but this logic missed the case where a filter has no references. For example, `WHERE false` is deterministic but is independent of the actual data.
This patch fixes this minor bug by adding a new check to ensure that we don't push filters beneath aggregates when those filters don't reference any columns.
## How was this patch tested?
New regression test in FilterPushdownSuite.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#15289 from JoshRosen/SPARK-17712.
## What changes were proposed in this pull request?
We added native versions of `collect_set` and `collect_list` in Spark 2.0. These currently also (try to) collect null values, this is different from the original Hive implementation. This PR fixes this by adding a null check to the `Collect.update` method.
## How was this patch tested?
Added a regression test to `DataFrameAggregateSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15208 from hvanhovell/SPARK-17641.
This patch ports changes from #15185 to Spark 2.x. In that patch, a correctness bug in Spark 1.6.x which was caused by an invalid `equals()` comparison between an `UnsafeRow` and another row of a different format. Spark 2.x is not affected by that specific correctness bug but it can still reap the error-prevention benefits of that patch's changes, which modify ``UnsafeRow.equals()` to throw an IllegalArgumentException if it is called with an object that is not an `UnsafeRow`.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#15265 from JoshRosen/SPARK-17618-master.
## What changes were proposed in this pull request?
This PR introduces more compact representation for ```UnsafeArrayData```.
```UnsafeArrayData``` needs to accept ```null``` value in each entry of an array. In the current version, it has three parts
```
[numElements] [offsets] [values]
```
`Offsets` has the number of `numElements`, and represents `null` if its value is negative. It may increase memory footprint, and introduces an indirection for accessing each of `values`.
This PR uses bitvectors to represent nullability for each element like `UnsafeRow`, and eliminates an indirection for accessing each element. The new ```UnsafeArrayData``` has four parts.
```
[numElements][null bits][values or offset&length][variable length portion]
```
In the `null bits` region, we store 1 bit per element, represents whether an element is null. Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries.
In the `values or offset&length` region, we store the content of elements. For fields that hold fixed-length primitive types, such as long, double, or int, we store the value directly in the field. For fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the base address of the array) that points to the beginning of the variable-length field and length (they are combined into a long). Each is word-aligned. For `variable length portion`, each is aligned to 8-byte boundaries.
The new format can reduce memory footprint and improve performance of accessing each element. An example of memory foot comparison:
1024x1024 elements integer array
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024 + 1024x1024 = 2M bytes
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024/8 + 1024x1024 = 1.25M bytes
In summary, we got 1.0-2.6x performance improvements over the code before applying this PR.
Here are performance results of [benchmark programs](04d2e4b6db/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala):
**Read UnsafeArrayData**: 1.7x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 430 / 436 390.0 2.6 1.0X
Double 456 / 485 367.8 2.7 0.9X
With SPARK-15962
Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 252 / 260 666.1 1.5 1.0X
Double 281 / 292 597.7 1.7 0.9X
````
**Write UnsafeArrayData**: 1.0x and 1.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 203 / 273 103.4 9.7 1.0X
Double 239 / 356 87.9 11.4 0.8X
With SPARK-15962
Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 196 / 249 107.0 9.3 1.0X
Double 227 / 367 92.3 10.8 0.9X
````
**Get primitive array from UnsafeArrayData**: 2.6x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 207 / 217 304.2 3.3 1.0X
Double 257 / 363 245.2 4.1 0.8X
With SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 151 / 198 415.8 2.4 1.0X
Double 214 / 394 293.6 3.4 0.7X
````
**Create UnsafeArrayData from primitive array**: 1.7x and 2.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 340 / 385 185.1 5.4 1.0X
Double 479 / 705 131.3 7.6 0.7X
With SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Int 206 / 211 306.0 3.3 1.0X
Double 232 / 406 271.6 3.7 0.9X
````
1.7x and 1.4x performance improvements in [```UDTSerializationBenchmark```](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala) over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Without SPARK-15962
VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
serialize 442 / 533 0.0 441927.1 1.0X
deserialize 217 / 274 0.0 217087.6 2.0X
With SPARK-15962
VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
serialize 265 / 318 0.0 265138.5 1.0X
deserialize 155 / 197 0.0 154611.4 1.7X
````
## How was this patch tested?
Added unit tests into ```UnsafeArraySuite```
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#13680 from kiszk/SPARK-15962.
## What changes were proposed in this pull request?
This pull request adds Scala/Java DataFrame API for null ordering (NULLS FIRST | LAST).
Also did some minor clean up for related code (e.g. incorrect indentation), and renamed "orderby-nulls-ordering.sql" to be consistent with existing test files.
## How was this patch tested?
Added a new test case in DataFrameSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Author: Xin Wu <xinwu@us.ibm.com>
Closes#15123 from petermaxlee/SPARK-17551.
## What changes were proposed in this pull request?
We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example:
```sql
select grp,
collect_list(col1),
count(distinct col2)
from tbl_a
group by 1
```
This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6).
## How was this patch tested?
Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15187 from hvanhovell/SPARK-17616.
## What changes were proposed in this pull request?
After #15054 , there is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks.
This PR also improves the `getTempViewOrPermanentTableMetadata` that is introduced in #15054 , to make the code simpler.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15160 from cloud-fan/exists.
## What changes were proposed in this pull request?
Floor()/Ceil() of decimal is implemented using changePrecision() by passing a rounding mode, but the rounding mode is not respected when the decimal is in compact mode (could fit within a Long).
This Update the changePrecision() to respect rounding mode, which could be ROUND_FLOOR, ROUND_CEIL, ROUND_HALF_UP, ROUND_HALF_EVEN.
## How was this patch tested?
Added regression tests.
Author: Davies Liu <davies@databricks.com>
Closes#15154 from davies/decimal_round.
## What changes were proposed in this pull request?
We substitute logical plan with CTE definitions in the analyzer rule CTESubstitution. A CTE definition can be used in the logical plan for multiple times, and its analyzed logical plan should be the same. We should not analyze CTE definitions multiple times when they are reused in the query.
By analyzing CTE definitions before substitution, we can support defining CTE in subquery.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#15146 from viirya/cte-analysis-once.
## What changes were proposed in this pull request?
Remainder(%) expression's `eval()` returns incorrect result when the dividend is a big double. The reason is that Remainder converts the double dividend to decimal to do "%", and that lose precision.
This bug only affects the `eval()` that is used by constant folding, the codegen path is not impacted.
### Before change
```
scala> -5083676433652386516D % 10
res2: Double = -6.0
scala> spark.sql("select -5083676433652386516D % 10 as a").show
+---+
| a|
+---+
|0.0|
+---+
```
### After change
```
scala> spark.sql("select -5083676433652386516D % 10 as a").show
+----+
| a|
+----+
|-6.0|
+----+
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#15171 from clockfly/SPARK-17617.
### What changes were proposed in this pull request?
- When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for partition-related ALTER TABLE commands. However, it always reports a confusing error message. For example,
```
Partition spec is invalid. The spec (a, b) must match the partition spec () defined in table '`testview`';
```
- When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for `ALTER TABLE ... UNSET TBLPROPERTIES`. However, it reports a missing table property. For example,
```
Attempted to unset non-existent property 'p' in table '`testView`';
```
- When `ANALYZE TABLE` is called on a view or a temporary view, we should issue an error message. However, it reports a strange error:
```
ANALYZE TABLE is not supported for Project
```
- When inserting into a temporary view that is generated from `Range`, we will get the following error message:
```
assertion failed: No plan for 'InsertIntoTable Range (0, 10, step=1, splits=Some(1)), false, false
+- Project [1 AS 1#20]
+- OneRowRelation$
```
This PR is to fix the above four issues.
### How was this patch tested?
Added multiple test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15054 from gatorsmile/tempViewDDL.
This patch addresses a corner-case escaping bug where field names which contain special characters were unsafely interpolated into error message string literals in generated Java code, leading to compilation errors.
This patch addresses these issues by using `addReferenceObj` to store the error messages as string fields rather than inline string constants.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#15156 from JoshRosen/SPARK-17160.
## What changes were proposed in this pull request?
In optimizer, we try to evaluate the condition to see whether it's nullable or not, but some expressions are not evaluable, we should check that before evaluate it.
## How was this patch tested?
Added regression tests.
Author: Davies Liu <davies@databricks.com>
Closes#15103 from davies/udf_join.
## What changes were proposed in this pull request?
In `ExpressionEvalHelper`, we check the equality between two double values by comparing whether the expected value is within the range [target - tolerance, target + tolerance], but this can cause a negative false when the compared numerics are very large.
Before:
```
val1 = 1.6358558070241E306
val2 = 1.6358558070240974E306
ExpressionEvalHelper.compareResults(val1, val2)
false
```
In fact, `val1` and `val2` are but with different precisions, we should tolerant this case by comparing with percentage range, eg.,expected is within range [target - target * tolerance_percentage, target + target * tolerance_percentage].
After:
```
val1 = 1.6358558070241E306
val2 = 1.6358558070240974E306
ExpressionEvalHelper.compareResults(val1, val2)
true
```
## How was this patch tested?
Exsiting testcases.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15059 from jiangxb1987/deq.
## What changes were proposed in this pull request?
In `SessionCatalog`, we have several operations(`tableExists`, `dropTable`, `loopupRelation`, etc) that handle both temp views and metastore tables/views. This brings some bugs to DDL commands that want to handle temp view only or metastore table/view only. These bugs are:
1. `CREATE TABLE USING` will fail if a same-name temp view exists
2. `Catalog.dropTempView`will un-cache and drop metastore table if a same-name table exists
3. `saveAsTable` will fail or have unexpected behaviour if a same-name temp view exists.
These bug fixes are pulled out from https://github.com/apache/spark/pull/14962 and targets both master and 2.0 branch
## How was this patch tested?
new regression tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#15099 from cloud-fan/fix-view.
## What changes were proposed in this pull request?
This PR fixes all the instances which was fixed in the previous PR.
To make sure, I manually debugged and also checked the Scala source. `length` in [LinearSeqOptimized.scala#L49-L57](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/LinearSeqOptimized.scala#L49-L57) is O(n). Also, `size` calls `length` via [SeqLike.scala#L106](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/SeqLike.scala#L106).
For debugging, I have created these as below:
```scala
ArrayBuffer(1, 2, 3)
Array(1, 2, 3)
List(1, 2, 3)
Seq(1, 2, 3)
```
and then called `size` and `length` for each to debug.
## How was this patch tested?
I ran the bash as below on Mac
```bash
find . -name *.scala -type f -exec grep -il "while (.*\\.length)" {} \; | grep "src/main"
find . -name *.scala -type f -exec grep -il "while (.*\\.size)" {} \; | grep "src/main"
```
and then checked each.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#15093 from HyukjinKwon/SPARK-17480-followup.
The existing code caches all stats for all columns for each partition
in the driver; for a large relation, this causes extreme memory usage,
which leads to gc hell and application failures.
It seems that only the size in bytes of the data is actually used in the
driver, so instead just colllect that. In executors, the full stats are
still kept, but that's not a big problem; we expect the data to be distributed
and thus not really incur in too much memory pressure in each individual
executor.
There are also potential improvements on the executor side, since the data
being stored currently is very wasteful (e.g. storing boxed types vs.
primitive types for stats). But that's a separate issue.
On a mildly related change, I'm also adding code to catch exceptions in the
code generator since Janino was breaking with the test data I tried this
patch on.
Tested with unit tests and by doing a count a very wide table (20k columns)
with many partitions.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#15112 from vanzin/SPARK-17549.
## What changes were proposed in this pull request?
This PR is a follow up of SPARK-17356. Current implementation of `TreeNode.toJSON` recursively converts all fields of TreeNode to JSON, even if the field is of type `Seq` or type Map. This may trigger out of memory exception in cases like:
1. the Seq or Map can be very big. Converting them to JSON may take huge memory, which may trigger out of memory error.
2. Some user space input may also be propagated to the Plan. The user space input can be of arbitrary type, and may also be self-referencing. Trying to print user space input to JSON may trigger out of memory error or stack overflow error.
For a code example, please check the Jira description of SPARK-17426.
In this PR, we refactor the `TreeNode.toJSON` so that we only convert a field to JSON string if the field is a safe type.
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14990 from clockfly/json_oom2.
## What changes were proposed in this pull request?
This change preserves aliases that are given for pivot aggregations
## How was this patch tested?
New unit test
Author: Andrew Ray <ray.andrew@gmail.com>
Closes#15111 from aray/SPARK-17458.
## What changes were proposed in this pull request?
The Antlr lexer we use to tokenize a SQL string may wrongly tokenize a fully qualified identifier as a decimal number token. For example, table identifier `default.123_table` is wrongly tokenized as
```
default // Matches lexer rule IDENTIFIER
.123 // Matches lexer rule DECIMAL_VALUE
_TABLE // Matches lexer rule IDENTIFIER
```
The correct tokenization for `default.123_table` should be:
```
default // Matches lexer rule IDENTIFIER,
. // Matches a single dot
123_TABLE // Matches lexer rule IDENTIFIER
```
This PR fix the Antlr grammar so that it can tokenize fully qualified identifier correctly:
1. Fully qualified table name can be parsed correctly. For example, `select * from database.123_suffix`.
2. Fully qualified column name can be parsed correctly, for example `select a.123_suffix from a`.
### Before change
#### Case 1: Failed to parse fully qualified column name
```
scala> spark.sql("select a.123_column from a").show
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input '.123' expecting {<EOF>,
...
, IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 8)
== SQL ==
select a.123_column from a
--------^^^
```
#### Case 2: Failed to parse fully qualified table name
```
scala> spark.sql("select * from default.123_table")
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input '.123' expecting {<EOF>,
...
IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 21)
== SQL ==
select * from default.123_table
---------------------^^^
```
### After Change
#### Case 1: fully qualified column name, no ParseException thrown
```
scala> spark.sql("select a.123_column from a").show
```
#### Case 2: fully qualified table name, no ParseException thrown
```
scala> spark.sql("select * from default.123_table")
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#15006 from clockfly/SPARK-17364.
## What changes were proposed in this pull request?
select length(11);
select length(2.0);
these sql will return errors, but hive is ok.
this PR will support casting input types implicitly for function length
the correct result is:
select length(11) return 2
select length(2.0) return 3
Author: 岑玉海 <261810726@qq.com>
Author: cenyuhai <cenyuhai@didichuxing.com>
Closes#15014 from cenyuhai/SPARK-17429.
## What changes were proposed in this pull request?
This PR fixes an issue with aggregates that have an empty input, and use a literals as their grouping keys. These aggregates are currently interpreted as aggregates **without** grouping keys, this triggers the ungrouped code path (which aways returns a single row).
This PR fixes the `RemoveLiteralFromGroupExpressions` optimizer rule, which changes the semantics of the Aggregate by eliminating all literal grouping keys.
## How was this patch tested?
Added tests to `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#15101 from hvanhovell/SPARK-17114-3.
## What changes were proposed in this pull request?
This PR has the appendRowUntilExceedingPageSize test in RowBasedKeyValueBatchSuite use whatever spark.buffer.pageSize value a user has specified to prevent a test failure for anyone testing Apache Spark on a box with a reduced page size. The test is currently hardcoded to use the default page size which is 64 MB so this minor PR is a test improvement
## How was this patch tested?
Existing unit tests with 1 MB page size and with 64 MB (the default) page size
Author: Adam Roberts <aroberts@uk.ibm.com>
Closes#15079 from a-roberts/patch-5.
## What changes were proposed in this pull request?
Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values.
However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC).
This PR is to support this new feature.
## How was this patch tested?
New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries.
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: Xin Wu <xinwu@us.ibm.com>
Closes#14842 from xwu0226/SPARK-10747.
### What changes were proposed in this pull request?
As explained in https://github.com/apache/spark/pull/14797:
>Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs.
For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result.
We should not optimize the query in CTAS more than once. For example,
```Scala
spark.range(99, 101).createOrReplaceTempView("tab1")
val sqlStmt = "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1"
sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt")
checkAnswer(spark.table("tab2"), sql(sqlStmt))
```
Before this PR, the results do not match
```
== Results ==
!== Correct Answer - 2 == == Spark Answer - 2 ==
![100,100.000000000000000000] [100,null]
[99,99.000000000000000000] [99,99.000000000000000000]
```
After this PR, the results match.
```
+---+----------------------+
|id |num |
+---+----------------------+
|99 |99.000000000000000000 |
|100|100.000000000000000000|
+---+----------------------+
```
In this PR, we do not treat the `query` in CTAS as a child. Thus, the `query` will not be optimized when optimizing CTAS statement. However, we still need to analyze it for normalizing and verifying the CTAS in the Analyzer. Thus, we do it in the analyzer rule `PreprocessDDL`, because so far only this rule needs the analyzed plan of the `query`.
### How was this patch tested?
Added a test
Author: gatorsmile <gatorsmile@gmail.com>
Closes#15048 from gatorsmile/ctasOptimized.
## What changes were proposed in this pull request?
In `ReorderAssociativeOperator` rule, we extract foldable expressions with Add/Multiply arithmetics, and replace with eval literal. For example, `(a + 1) + (b + 2)` is optimized to `(a + b + 3)` by this rule.
For aggregate operator, output expressions should be derived from groupingExpressions, current implemenation of `ReorderAssociativeOperator` rule may break this promise. A instance could be:
```
SELECT
((t1.a + 1) + (t2.a + 2)) AS out_col
FROM
testdata2 AS t1
INNER JOIN
testdata2 AS t2
ON
(t1.a = t2.a)
GROUP BY (t1.a + 1), (t2.a + 2)
```
`((t1.a + 1) + (t2.a + 2))` is optimized to `(t1.a + t2.a + 3)`, which could not be derived from `ExpressionSet((t1.a +1), (t2.a + 2))`.
Maybe we should improve the rule of `ReorderAssociativeOperator` by adding a GroupingExpressionSet to keep Aggregate.groupingExpressions, and respect these expressions during the optimize stage.
## How was this patch tested?
Add new test case in `ReorderAssociativeOperatorSuite`.
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#14917 from jiangxb1987/rao.
## What changes were proposed in this pull request?
This PR build on #14976 and fixes a correctness bug that would cause the wrong quantile to be returned for small target errors.
## How was this patch tested?
This PR adds 8 unit tests that were failing without the fix.
Author: Timothy Hunter <timhunter@databricks.com>
Author: Sean Owen <sowen@cloudera.com>
Closes#15002 from thunterdb/ml-1783.
## What changes were proposed in this pull request?
Before this change, we would always allocate 64MB per aggregation task for the first-level hash map storage, even when running in low-memory situations such as local mode. This changes it to use the memory manager default page size, which is automatically reduced from 64MB in these situations.
cc ooq JoshRosen
## How was this patch tested?
Tested manually with `bin/spark-shell --master=local[32]` and verifying that `(1 to math.pow(10, 3).toInt).toDF("n").withColumn("m", 'n % 2).groupBy('m).agg(sum('n)).show` does not crash.
Author: Eric Liang <ekl@databricks.com>
Closes#15016 from ericl/sc-4483.
## What changes were proposed in this pull request?
Fixing the typo in the unit test of CodeGenerationSuite.scala
## How was this patch tested?
Ran the unit test after fixing the typo and it passes
Author: Srinivasa Reddy Vundela <vsr@cloudera.com>
Closes#14989 from vundela/typo_fix.
## What changes were proposed in this pull request?
`select size(null)` returns -1 in Hive. In order to be compatible, we should return `-1`.
## How was this patch tested?
unit test in `CollectionFunctionsSuite` and `DataFrameFunctionsSuite`.
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#14991 from adrian-wang/size.
## What changes were proposed in this pull request?
We should generally use `ArrayBuffer.+=(A)` rather than `ArrayBuffer.append(A)`, because `append(A)` would involve extra boxing / unboxing.
## How was this patch tested?
N/A
Author: Liwei Lin <lwlin7@gmail.com>
Closes#14914 from lw-lin/append_to_plus_eq_v2.
## What changes were proposed in this pull request?
Join processing in the parser relies on the fact that the grammar produces a right nested trees, for instance the parse tree for `select * from a join b join c` is expected to produce a tree similar to `JOIN(a, JOIN(b, c))`. However there are cases in which this (invariant) is violated, like:
```sql
SELECT COUNT(1)
FROM test T1
CROSS JOIN test T2
JOIN test T3
ON T3.col = T1.col
JOIN test T4
ON T4.col = T1.col
```
In this case the parser returns a tree in which Joins are located on both the left and the right sides of the parent join node.
This PR introduces a different grammar rule which does not make this assumption. The new rule takes a relation and searches for zero or more joined relations. As a bonus processing is much easier.
## How was this patch tested?
Existing tests and I have added a regression test to the plan parser suite.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14867 from hvanhovell/SPARK-17296.
## What changes were proposed in this pull request?
class `org.apache.spark.sql.types.Metadata` is widely used in mllib to store some ml attributes. `Metadata` is commonly stored in `Alias` expression.
```
case class Alias(child: Expression, name: String)(
val exprId: ExprId = NamedExpression.newExprId,
val qualifier: Option[String] = None,
val explicitMetadata: Option[Metadata] = None,
override val isGenerated: java.lang.Boolean = false)
```
The `Metadata` can take a big memory footprint since the number of attributes is big ( in scale of million). When `toJSON` is called on `Alias` expression, the `Metadata` will also be converted to a big JSON string.
If a plan contains many such kind of `Alias` expressions, it may trigger out of memory error when `toJSON` is called, since converting all `Metadata` references to JSON will take huge memory.
With this PR, we will skip scanning Metadata when doing JSON conversion. For a reproducer of the OOM, and analysis, please look at jira https://issues.apache.org/jira/browse/SPARK-17356.
## How was this patch tested?
Existing tests.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14915 from clockfly/json_oom.
## What changes were proposed in this pull request?
Using the public `Catalog` API, users can create a file-based data source table, without giving the path options. For this case, currently we can create the table successfully, but fail when we read it. Ideally we should fail during creation.
This is because when we create data source table, we resolve the data source relation without validating path: `resolveRelation(checkPathExist = false)`.
Looking back to why we add this trick(`checkPathExist`), it's because when we call `resolveRelation` for managed table, we add the path to data source options but the path is not created yet. So why we add this not-yet-created path to data source options? This PR fix the problem by adding path to options after we call `resolveRelation`. Then we can remove the `checkPathExist` parameter in `DataSource.resolveRelation` and do some related cleanups.
## How was this patch tested?
existing tests and new test in `CatalogSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14921 from cloud-fan/check-path.
## What changes were proposed in this pull request?
If `ScalaUDF` throws exceptions during executing user code, sometimes it's hard for users to figure out what's wrong, especially when they use Spark shell. An example
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 12 in stage 325.0 failed 4 times, most recent failure: Lost task 12.3 in stage 325.0 (TID 35622, 10.0.207.202): java.lang.NullPointerException
at line8414e872fb8b42aba390efc153d1611a12.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$2.apply(<console>:40)
at line8414e872fb8b42aba390efc153d1611a12.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$2.apply(<console>:40)
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
...
```
We should catch these exceptions and rethrow them with better error message, to say that the exception is happened in scala udf.
This PR also does some clean up for `ScalaUDF` and add a unit test suite for it.
## How was this patch tested?
the new test suite
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14850 from cloud-fan/npe.
## What changes were proposed in this pull request?
1. Support generation table-level statistics for
- hive tables in HiveExternalCatalog
- data source tables in HiveExternalCatalog
- data source tables in InMemoryCatalog.
2. Add a property "catalogStats" in CatalogTable to hold statistics in Spark side.
3. Put logics of statistics transformation between Spark and Hive in HiveClientImpl.
4. Extend Statistics class by adding rowCount (will add estimatedSize when we have column stats).
## How was this patch tested?
add unit tests
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wangzhenhua@huawei.com>
Closes#14712 from wzhfy/tableStats.
## What changes were proposed in this pull request?
It's really weird that we allow users to specify database in both from table name and to table name
in `ALTER TABLE RENAME TO`, while logically we can't support rename a table to a different database.
Both postgres and MySQL disallow this syntax, it's reasonable to follow them and simply our code.
## How was this patch tested?
new test in `DDLCommandSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14955 from cloud-fan/rename.
## What changes were proposed in this pull request?
Improved the code quality of spark by replacing all pattern match on boolean value by if/else block.
## How was this patch tested?
By running the tests
Author: Shivansh <shiv4nsh@gmail.com>
Closes#14873 from shiv4nsh/SPARK-17308.
### What changes were proposed in this pull request?
This is another step to get rid of HiveClient from `HiveSessionState`. All the metastore interactions should be through `ExternalCatalog` interface. However, the existing implementation of `InsertIntoHiveTable ` still requires Hive clients. This PR is to remove HiveClient by moving the metastore interactions into `ExternalCatalog`.
### How was this patch tested?
Existing test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14888 from gatorsmile/removeClientFromInsertIntoHiveTable.
## What changes were proposed in this pull request?
the `catalogString` for `ArrayType` and `MapType` currently calls the `simpleString` method on its children. This is a problem when the child is a struct, the `struct.simpleString` implementation truncates the number of fields it shows (25 at max). This breaks the generation of a proper `catalogString`, and has shown to cause errors while writing to Hive.
This PR fixes this by providing proper `catalogString` implementations for `ArrayData` or `MapData`.
## How was this patch tested?
Added testing for `catalogString` to `DataTypeSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14938 from hvanhovell/SPARK-17335.
## What changes were proposed in this pull request?
Require the use of CROSS join syntax in SQL (and a new crossJoin
DataFrame API) to specify explicit cartesian products between relations.
By cartesian product we mean a join between relations R and S where
there is no join condition involving columns from both R and S.
If a cartesian product is detected in the absence of an explicit CROSS
join, an error must be thrown. Turning on the
"spark.sql.crossJoin.enabled" configuration flag will disable this check
and allow cartesian products without an explicit CROSS join.
The new crossJoin DataFrame API must be used to specify explicit cross
joins. The existing join(DataFrame) method will produce a INNER join
that will require a subsequent join condition.
That is df1.join(df2) is equivalent to select * from df1, df2.
## How was this patch tested?
Added cross-join.sql to the SQLQueryTestSuite to test the check for cartesian products. Added a couple of tests to the DataFrameJoinSuite to test the crossJoin API. Modified various other test suites to explicitly specify a cross join where an INNER join or a comma-separated list was previously used.
Author: Srinath Shankar <srinath@databricks.com>
Closes#14866 from srinathshankar/crossjoin.
### What changes were proposed in this pull request?
Function-related `HiveExternalCatalog` APIs do not have enough verification logics. After the PR, `HiveExternalCatalog` and `InMemoryCatalog` become consistent in the error handling.
For example, below is the exception we got when calling `renameFunction`.
```
15:13:40.369 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database db1, returning NoSuchObjectException
15:13:40.377 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database db2, returning NoSuchObjectException
15:13:40.739 ERROR DataNucleus.Datastore.Persist: Update of object "org.apache.hadoop.hive.metastore.model.MFunction205629e9" using statement "UPDATE FUNCS SET FUNC_NAME=? WHERE FUNC_ID=?" failed : org.apache.derby.shared.common.error.DerbySQLIntegrityConstraintViolationException: The statement was aborted because it would have caused a duplicate key value in a unique or primary key constraint or unique index identified by 'UNIQUEFUNCTION' defined on 'FUNCS'.
at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.Util.generateCsSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.TransactionResourceImpl.wrapInSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.TransactionResourceImpl.handleException(Unknown Source)
```
### How was this patch tested?
Improved the existing test cases to check whether the messages are right.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14521 from gatorsmile/functionChecking.
## What changes were proposed in this pull request?
This PR is the second 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 `RowBatch`. We then automatically pick between the two implementations based on certain knobs.
In this second-step PR, we enable `RowBasedHashMapGenerator` in `HashAggregateExec`.
## How was this patch tested?
Added tests: `RowBasedAggregateHashMapSuite` and ` VectorizedAggregateHashMapSuite`
Additional micro-benchmarks tests and TPCDS results will be added in a separate PR in the series.
Author: Qifan Pu <qifan.pu@gmail.com>
Author: ooq <qifan.pu@gmail.com>
Closes#14176 from ooq/rowbasedfastaggmap-pr2.
## What changes were proposed in this pull request?
Some codes in subexpressionEliminationForWholeStageCodegen are never used actually.
Remove them using this PR.
## How was this patch tested?
Local unit tests.
Author: Yucai Yu <yucai.yu@intel.com>
Closes#14366 from yucai/subExpr_unused_codes.
## What changes were proposed in this pull request?
Avoid allocating some 0-length arrays, esp. in UTF8String, and by using Array.empty in Scala over Array[T]()
## How was this patch tested?
Jenkins
Author: Sean Owen <sowen@cloudera.com>
Closes#14895 from srowen/SPARK-17331.
## What changes were proposed in this pull request?
This PR adds the ability to parse SQL (hexadecimal) binary literals (AKA bit strings). It follows the following syntax `X'[Hexadecimal Characters]+'`, for example: `X'01AB'` would create a binary the following binary array `0x01AB`.
If an uneven number of hexadecimal characters is passed, then the upper 4 bits of the initial byte are kept empty, and the lower 4 bits are filled using the first character. For example `X'1C7'` would create the following binary array `0x01C7`.
Binary data (Array[Byte]) does not have a proper `hashCode` and `equals` functions. This meant that comparing `Literal`s containing binary data was a pain. I have updated Literal.hashCode and Literal.equals to deal properly with binary data.
## How was this patch tested?
Added tests to the `ExpressionParserSuite`, `SQLQueryTestSuite` and `ExpressionSQLBuilderSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14832 from hvanhovell/SPARK-17263.
## What changes were proposed in this pull request?
Removing `semanticEquals()` from `SortOrder` because it can use the `semanticEquals()` provided by its parent class (`Expression`). This was as per suggestion by cloud-fan at 7192418b3a (r77106801)
## How was this patch tested?
Ran the test added in https://github.com/apache/spark/pull/14841
Author: Tejas Patil <tejasp@fb.com>
Closes#14910 from tejasapatil/SPARK-17271_remove_semantic_ordering.
## What changes were proposed in this pull request?
This PR implements aggregation function `percentile_approx`. Function `percentile_approx` returns the approximate percentile(s) of a column at the given percentage(s). A percentile is a watermark value below which a given percentage of the column values fall. For example, the percentile of column `col` at percentage 50% is the median value of column `col`.
### Syntax:
```
# Returns percentile at a given percentage value. The approximation error can be reduced by increasing parameter accuracy, at the cost of memory.
percentile_approx(col, percentage [, accuracy])
# Returns percentile value array at given percentage value array
percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy])
```
### Features:
1. This function supports partial aggregation.
2. The memory consumption is bounded. The larger `accuracy` parameter we choose, we smaller error we get. The default accuracy value is 10000, to match with Hive default setting. Choose a smaller value for smaller memory footprint.
3. This function supports window function aggregation.
### Example usages:
```
## Returns the 25th percentile value, with default accuracy
SELECT percentile_approx(col, 0.25) FROM table
## Returns an array of percentile value (25th, 50th, 75th), with default accuracy
SELECT percentile_approx(col, array(0.25, 0.5, 0.75)) FROM table
## Returns 25th percentile value, with custom accuracy value 100, larger accuracy parameter yields smaller approximation error
SELECT percentile_approx(col, 0.25, 100) FROM table
## Returns the 25th, and 50th percentile values, with custom accuracy value 100
SELECT percentile_approx(col, array(0.25, 0.5), 100) FROM table
```
### NOTE:
1. The `percentile_approx` implementation is different from Hive, so the result returned on same query maybe slightly different with Hive. This implementation uses `QuantileSummaries` as the underlying probabilistic data structure, and mainly follows paper `Space-efficient Online Computation of Quantile Summaries` by Greenwald, Michael and Khanna, Sanjeev. (http://dx.doi.org/10.1145/375663.375670)`
2. The current implementation of `QuantileSummaries` doesn't support automatic compression. This PR has a rule to do compression automatically at the caller side, but it may not be optimal.
## How was this patch tested?
Unit test, and Sql query test.
## Acknowledgement
1. This PR's work in based on lw-lin's PR https://github.com/apache/spark/pull/14298, with improvements like supporting partial aggregation, fixing out of memory issue.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14868 from clockfly/appro_percentile_try_2.
## What changes were proposed in this pull request?
This PR eliminates redundant cast from an `ArrayType` with `containsNull = false` or a `MapType` with `containsNull = false`.
For example, in `ArrayType` case, current implementation leaves a cast `cast(value#63 as array<double>).toDoubleArray`. However, we can eliminate `cast(value#63 as array<double>)` if we know `value#63` does not include `null`. This PR apply this elimination for `ArrayType` and `MapType` in `SimplifyCasts` at a plan optimization phase.
In summary, we got 1.2-1.3x performance improvements over the code before applying this PR.
Here are performance results of benchmark programs:
```
test("Read array in Dataset") {
import sparkSession.implicits._
val iters = 5
val n = 1024 * 1024
val rows = 15
val benchmark = new Benchmark("Read primnitive array", n)
val rand = new Random(511)
val intDS = sparkSession.sparkContext.parallelize(0 until rows, 1)
.map(i => Array.tabulate(n)(i => i)).toDS()
intDS.count() // force to create ds
val lastElement = n - 1
val randElement = rand.nextInt(lastElement)
benchmark.addCase(s"Read int array in Dataset", numIters = iters)(iter => {
val idx0 = randElement
val idx1 = lastElement
intDS.map(a => a(0) + a(idx0) + a(idx1)).collect
})
val doubleDS = sparkSession.sparkContext.parallelize(0 until rows, 1)
.map(i => Array.tabulate(n)(i => i.toDouble)).toDS()
doubleDS.count() // force to create ds
benchmark.addCase(s"Read double array in Dataset", numIters = iters)(iter => {
val idx0 = randElement
val idx1 = lastElement
doubleDS.map(a => a(0) + a(idx0) + a(idx1)).collect
})
benchmark.run()
}
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4
Intel(R) Core(TM) i5-5257U CPU 2.70GHz
without this PR
Read primnitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Read int array in Dataset 525 / 690 2.0 500.9 1.0X
Read double array in Dataset 947 / 1209 1.1 902.7 0.6X
with this PR
Read primnitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
Read int array in Dataset 400 / 492 2.6 381.5 1.0X
Read double array in Dataset 788 / 870 1.3 751.4 0.5X
```
An example program that originally caused this performance issue.
```
val ds = Seq(Array(1.0, 2.0, 3.0), Array(4.0, 5.0, 6.0)).toDS()
val ds2 = ds.map(p => {
var s = 0.0
for (i <- 0 to 2) { s += p(i) }
s
})
ds2.show
ds2.explain(true)
```
Plans before this PR
```
== Parsed Logical Plan ==
'SerializeFromObject [input[0, double, true] AS value#68]
+- 'MapElements <function1>, obj#67: double
+- 'DeserializeToObject unresolveddeserializer(upcast(getcolumnbyordinal(0, ArrayType(DoubleType,false)), ArrayType(DoubleType,false), - root class: "scala.Array").toDoubleArray), obj#66: [D
+- LocalRelation [value#63]
== Analyzed Logical Plan ==
value: double
SerializeFromObject [input[0, double, true] AS value#68]
+- MapElements <function1>, obj#67: double
+- DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D
+- LocalRelation [value#63]
== Optimized Logical Plan ==
SerializeFromObject [input[0, double, true] AS value#68]
+- MapElements <function1>, obj#67: double
+- DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D
+- LocalRelation [value#63]
== Physical Plan ==
*SerializeFromObject [input[0, double, true] AS value#68]
+- *MapElements <function1>, obj#67: double
+- *DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D
+- LocalTableScan [value#63]
```
Plans after this PR
```
== Parsed Logical Plan ==
'SerializeFromObject [input[0, double, true] AS value#6]
+- 'MapElements <function1>, obj#5: double
+- 'DeserializeToObject unresolveddeserializer(upcast(getcolumnbyordinal(0, ArrayType(DoubleType,false)), ArrayType(DoubleType,false), - root class: "scala.Array").toDoubleArray), obj#4: [D
+- LocalRelation [value#1]
== Analyzed Logical Plan ==
value: double
SerializeFromObject [input[0, double, true] AS value#6]
+- MapElements <function1>, obj#5: double
+- DeserializeToObject cast(value#1 as array<double>).toDoubleArray, obj#4: [D
+- LocalRelation [value#1]
== Optimized Logical Plan ==
SerializeFromObject [input[0, double, true] AS value#6]
+- MapElements <function1>, obj#5: double
+- DeserializeToObject value#1.toDoubleArray, obj#4: [D
+- LocalRelation [value#1]
== Physical Plan ==
*SerializeFromObject [input[0, double, true] AS value#6]
+- *MapElements <function1>, obj#5: double
+- *DeserializeToObject value#1.toDoubleArray, obj#4: [D
+- LocalTableScan [value#1]
```
## How was this patch tested?
Tested by new test cases in `SimplifyCastsSuite`
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#13704 from kiszk/SPARK-15985.
### What changes were proposed in this pull request?
Hive Index tables are not supported by Spark SQL. Thus, we issue an exception when users try to access Hive Index tables. When the internal function `tableExists` tries to access Hive Index tables, it always gets the same error message: ```Hive index table is not supported```. This message could be confusing to users, since their SQL operations could be completely unrelated to Hive Index tables. For example, when users try to alter a table to a new name and there exists an index table with the same name, the expected exception should be a `TableAlreadyExistsException`.
This PR made the following changes:
- Introduced a new `AnalysisException` type: `SQLFeatureNotSupportedException`. When users try to access an `Index Table`, we will issue a `SQLFeatureNotSupportedException`.
- `tableExists` returns `true` when hitting a `SQLFeatureNotSupportedException` and the feature is `Hive index table`.
- Add a checking `requireTableNotExists` for `SessionCatalog`'s `createTable` API; otherwise, the current implementation relies on the Hive's internal checking.
### How was this patch tested?
Added a test case
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14801 from gatorsmile/tableExists.
There's an unused `classTag` val in the AtomicType base class which is causing unnecessary slowness in deserialization because it needs to grab ScalaReflectionLock and create a new runtime reflection mirror. Removing this unused code gives a small but measurable performance boost in SQL task deserialization.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#14869 from JoshRosen/remove-unused-classtag.
## What changes were proposed in this pull request?
This PR split the the single `createPartitions()` call into smaller batches, which could prevent Hive metastore from OOM (caused by millions of partitions).
It will also try to gather all the fast stats (number of files and total size of all files) in parallel to avoid the bottle neck of listing the files in metastore sequential, which is controlled by spark.sql.gatherFastStats (enabled by default).
## How was this patch tested?
Tested locally with 10000 partitions and 100 files with embedded metastore, without gathering fast stats in parallel, adding partitions took 153 seconds, after enable that, gathering the fast stats took about 34 seconds, adding these partitions took 25 seconds (most of the time spent in object store), 59 seconds in total, 2.5X faster (with larger cluster, gathering will much faster).
Author: Davies Liu <davies@databricks.com>
Closes#14607 from davies/repair_batch.
## What changes were proposed in this pull request?
Jira : https://issues.apache.org/jira/browse/SPARK-17271
Planner is adding un-needed SORT operation due to bug in the way comparison for `SortOrder` is done at https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala#L253
`SortOrder` needs to be compared semantically because `Expression` within two `SortOrder` can be "semantically equal" but not literally equal objects.
eg. In case of `sql("SELECT * FROM table1 a JOIN table2 b ON a.col1=b.col1")`
Expression in required SortOrder:
```
AttributeReference(
name = "col1",
dataType = LongType,
nullable = false
) (exprId = exprId,
qualifier = Some("a")
)
```
Expression in child SortOrder:
```
AttributeReference(
name = "col1",
dataType = LongType,
nullable = false
) (exprId = exprId)
```
Notice that the output column has a qualifier but the child attribute does not but the inherent expression is the same and hence in this case we can say that the child satisfies the required sort order.
This PR includes following changes:
- Added a `semanticEquals` method to `SortOrder` so that it can compare underlying child expressions semantically (and not using default Object.equals)
- Fixed `EnsureRequirements` to use semantic comparison of SortOrder
## How was this patch tested?
- Added a test case to `PlannerSuite`. Ran rest tests in `PlannerSuite`
Author: Tejas Patil <tejasp@fb.com>
Closes#14841 from tejasapatil/SPARK-17271_sort_order_equals_bug.
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various join rules into a single file.
## How was this patch tested?
This should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#14846 from rxin/SPARK-17274.
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various expression optimization rules into a single file.
## How was this patch tested?
This should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#14845 from rxin/SPARK-17273.
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various subquery rules into a single file.
## How was this patch tested?
This should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#14844 from rxin/SPARK-17272.
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various finish analysis optimization stage rules into a single file. I'm submitting separate pull requests so we can more easily merge this in branch-2.0 to simplify optimizer backports.
## How was this patch tested?
This should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#14838 from rxin/SPARK-17269.
## What changes were proposed in this pull request?
As part of breaking Optimizer.scala apart, this patch moves various Dataset object optimization rules into a single file. I'm submitting separate pull requests so we can more easily merge this in branch-2.0 to simplify optimizer backports.
## How was this patch tested?
This should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes#14839 from rxin/SPARK-17270.
## What changes were proposed in this pull request?
Given that non-deterministic expressions can be stateful, pushing them down the query plan during the optimization phase can cause incorrect behavior. This patch fixes that issue by explicitly disabling that.
## How was this patch tested?
A new test in `FilterPushdownSuite` that checks catalyst behavior for both deterministic and non-deterministic join conditions.
Author: Sameer Agarwal <sameerag@cs.berkeley.edu>
Closes#14815 from sameeragarwal/constraint-inputfile.
## What changes were proposed in this pull request?
This PR adds parser support for `BigDecimal` literals. If you append the suffix `BD` to a valid number then this will be interpreted as a `BigDecimal`, for example `12.0E10BD` will interpreted into a BigDecimal with scale -9 and precision 3. This is useful in situations where you need exact values.
## How was this patch tested?
Added tests to `ExpressionParserSuite`, `ExpressionSQLBuilderSuite` and `SQLQueryTestSuite`.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14819 from hvanhovell/SPARK-17246.
## What changes were proposed in this pull request?
improve the document to make it easier to understand and also mention window operator.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14822 from cloud-fan/object-agg.
## What changes were proposed in this pull request?
Currently, type-widening does not work between `TimestampType` and `DateType`.
This applies to `SetOperation`, `Union`, `In`, `CaseWhen`, `Greatest`, `Leatest`, `CreateArray`, `CreateMap`, `Coalesce`, `NullIf`, `IfNull`, `Nvl` and `Nvl2`, .
This PR adds the support for widening `DateType` to `TimestampType` for them.
For a simple example,
**Before**
```scala
Seq(Tuple2(new Timestamp(0), new Date(0))).toDF("a", "b").selectExpr("greatest(a, b)").show()
```
shows below:
```
cannot resolve 'greatest(`a`, `b`)' due to data type mismatch: The expressions should all have the same type, got GREATEST(timestamp, date)
```
or union as below:
```scala
val a = Seq(Tuple1(new Timestamp(0))).toDF()
val b = Seq(Tuple1(new Date(0))).toDF()
a.union(b).show()
```
shows below:
```
Union can only be performed on tables with the compatible column types. DateType <> TimestampType at the first column of the second table;
```
**After**
```scala
Seq(Tuple2(new Timestamp(0), new Date(0))).toDF("a", "b").selectExpr("greatest(a, b)").show()
```
shows below:
```
+----------------------------------------------------+
|greatest(CAST(a AS TIMESTAMP), CAST(b AS TIMESTAMP))|
+----------------------------------------------------+
| 1969-12-31 16:00:...|
+----------------------------------------------------+
```
or union as below:
```scala
val a = Seq(Tuple1(new Timestamp(0))).toDF()
val b = Seq(Tuple1(new Date(0))).toDF()
a.union(b).show()
```
shows below:
```
+--------------------+
| _1|
+--------------------+
|1969-12-31 16:00:...|
|1969-12-31 00:00:...|
+--------------------+
```
## How was this patch tested?
Unit tests in `TypeCoercionSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Author: HyukjinKwon <gurwls223@gmail.com>
Closes#14786 from HyukjinKwon/SPARK-17212.
## What changes were proposed in this pull request?
This PR introduces an abstract class `TypedImperativeAggregate` so that an aggregation function of TypedImperativeAggregate can use **arbitrary** user-defined Java object as intermediate aggregation buffer object.
**This has advantages like:**
1. It now can support larger category of aggregation functions. For example, it will be much easier to implement aggregation function `percentile_approx`, which has a complex aggregation buffer definition.
2. It can be used to avoid doing serialization/de-serialization for every call of `update` or `merge` when converting domain specific aggregation object to internal Spark-Sql storage format.
3. It is easier to integrate with other existing monoid libraries like algebird, and supports more aggregation functions with high performance.
Please see `org.apache.spark.sql.TypedImperativeAggregateSuite.TypedMaxAggregate` to find an example of how to defined a `TypedImperativeAggregate` aggregation function.
Please see Java doc of `TypedImperativeAggregate` and Jira ticket SPARK-17187 for more information.
## How was this patch tested?
Unit tests.
Author: Sean Zhong <seanzhong@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#14753 from clockfly/object_aggregation_buffer_try_2.
This patch updates `Literal.sql` to properly generate SQL for `NaN` and `Infinity` float and double literals: these special values need to be handled differently from regular values, since simply appending a suffix to the value's `toString()` representation will not work for these values.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#14777 from JoshRosen/SPARK-17205.
### What changes were proposed in this pull request?
This PR is to fix an incorrect outer join elimination when filter's `isNotNull` constraints is unable to filter out all null-supplying rows. For example, `isnotnull(coalesce(b#227, c#238))`.
Users can hit this error when they try to use `using/natural outer join`, which is converted to a normal outer join with a `coalesce` expression on the `using columns`. For example,
```Scala
val a = Seq((1, 2), (2, 3)).toDF("a", "b")
val b = Seq((2, 5), (3, 4)).toDF("a", "c")
val c = Seq((3, 1)).toDF("a", "d")
val ab = a.join(b, Seq("a"), "fullouter")
ab.join(c, "a").explain(true)
```
The dataframe `ab` is doing `using full-outer join`, which is converted to a normal outer join with a `coalesce` expression. Constraints inference generates a `Filter` with constraints `isnotnull(coalesce(b#227, c#238))`. Then, it triggers a wrong outer join elimination and generates a wrong result.
```
Project [a#251, b#227, c#237, d#247]
+- Join Inner, (a#251 = a#246)
:- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237]
: +- Join FullOuter, (a#226 = a#236)
: :- Project [_1#223 AS a#226, _2#224 AS b#227]
: : +- LocalRelation [_1#223, _2#224]
: +- Project [_1#233 AS a#236, _2#234 AS c#237]
: +- LocalRelation [_1#233, _2#234]
+- Project [_1#243 AS a#246, _2#244 AS d#247]
+- LocalRelation [_1#243, _2#244]
== Optimized Logical Plan ==
Project [a#251, b#227, c#237, d#247]
+- Join Inner, (a#251 = a#246)
:- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237]
: +- Filter isnotnull(coalesce(a#226, a#236))
: +- Join FullOuter, (a#226 = a#236)
: :- LocalRelation [a#226, b#227]
: +- LocalRelation [a#236, c#237]
+- LocalRelation [a#246, d#247]
```
**A note to the `Committer`**, please also give the credit to dongjoon-hyun who submitted another PR for fixing this issue. https://github.com/apache/spark/pull/14580
### How was this patch tested?
Added test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14661 from gatorsmile/fixOuterJoinElimination.
## What changes were proposed in this pull request?
Currently `MapObjects` does not make copies of unsafe-backed data, leading to problems like [SPARK-17061](https://issues.apache.org/jira/browse/SPARK-17061) [SPARK-17093](https://issues.apache.org/jira/browse/SPARK-17093).
This patch makes `MapObjects` make copies of unsafe-backed data.
Generated code - prior to this patch:
```java
...
/* 295 */ if (isNull12) {
/* 296 */ convertedArray1[loopIndex1] = null;
/* 297 */ } else {
/* 298 */ convertedArray1[loopIndex1] = value12;
/* 299 */ }
...
```
Generated code - after this patch:
```java
...
/* 295 */ if (isNull12) {
/* 296 */ convertedArray1[loopIndex1] = null;
/* 297 */ } else {
/* 298 */ convertedArray1[loopIndex1] = value12 instanceof UnsafeRow? value12.copy() : value12;
/* 299 */ }
...
```
## How was this patch tested?
Add a new test case which would fail without this patch.
Author: Liwei Lin <lwlin7@gmail.com>
Closes#14698 from lw-lin/mapobjects-copy.
### What changes were proposed in this pull request?
Since `HiveClient` is used to interact with the Hive metastore, it should be hidden in `HiveExternalCatalog`. After moving `HiveClient` into `HiveExternalCatalog`, `HiveSharedState` becomes a wrapper of `HiveExternalCatalog`. Thus, removal of `HiveSharedState` becomes straightforward. After removal of `HiveSharedState`, the reflection logic is directly applied on the choice of `ExternalCatalog` types, based on the configuration of `CATALOG_IMPLEMENTATION`.
~~`HiveClient` is also used/invoked by the other entities besides HiveExternalCatalog, we defines the following two APIs: getClient and getNewClient~~
### How was this patch tested?
The existing test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14757 from gatorsmile/removeHiveClient.
## What changes were proposed in this pull request?
Given that filters based on non-deterministic constraints shouldn't be pushed down in the query plan, unnecessarily inferring them is confusing and a source of potential bugs. This patch simplifies the inferring logic by simply ignoring them.
## How was this patch tested?
Added a new test in `ConstraintPropagationSuite`.
Author: Sameer Agarwal <sameerag@cs.berkeley.edu>
Closes#14795 from sameeragarwal/deterministic-constraints.
## What changes were proposed in this pull request?
Currently, two-word window functions like `row_number`, `dense_rank`, `percent_rank`, and `cume_dist` are expressed without `_` in error messages. We had better show the correct names.
**Before**
```scala
scala> sql("select row_number()").show
java.lang.UnsupportedOperationException: Cannot evaluate expression: rownumber()
```
**After**
```scala
scala> sql("select row_number()").show
java.lang.UnsupportedOperationException: Cannot evaluate expression: row_number()
```
## How was this patch tested?
Pass the Jenkins and manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14571 from dongjoon-hyun/SPARK-16983.
## What changes were proposed in this pull request?
Actually Spark SQL doesn't support index, the catalog table type `INDEX` is from Hive. However, most operations in Spark SQL can't handle index table, e.g. create table, alter table, etc.
Logically index table should be invisible to end users, and Hive also generates special table name for index table to avoid users accessing it directly. Hive has special SQL syntax to create/show/drop index tables.
At Spark SQL side, although we can describe index table directly, but the result is unreadable, we should use the dedicated SQL syntax to do it(e.g. `SHOW INDEX ON tbl`). Spark SQL can also read index table directly, but the result is always empty.(Can hive read index table directly?)
This PR remove the table type `INDEX`, to make it clear that Spark SQL doesn't support index currently.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14752 from cloud-fan/minor2.
When Spark emits SQL for a string literal, it should wrap the string in single quotes, not double quotes. Databases which adhere more strictly to the ANSI SQL standards, such as Postgres, allow only single-quotes to be used for denoting string literals (see http://stackoverflow.com/a/1992331/590203).
Author: Josh Rosen <joshrosen@databricks.com>
Closes#14763 from JoshRosen/SPARK-17194.
## What changes were proposed in this pull request?
Use `CatalystConf.resolver` consistently for case-sensitivity comparison (removed dups).
## How was this patch tested?
Local build. Waiting for Jenkins to ensure clean build and test.
Author: Jacek Laskowski <jacek@japila.pl>
Closes#14771 from jaceklaskowski/17199-catalystconf-resolver.
## What changes were proposed in this pull request?
This is a sub-task of [SPARK-16283](https://issues.apache.org/jira/browse/SPARK-16283) (Implement percentile_approx SQL function), which moves class QuantileSummaries to project catalyst so that it can be reused when implementing aggregation function `percentile_approx`.
## How was this patch tested?
This PR only does class relocation, class implementation is not changed.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14754 from clockfly/move_QuantileSummaries_to_catalyst.
## What changes were proposed in this pull request?
This PR marks the abstract class `Collect` as non-deterministic since the results of `CollectList` and `CollectSet` depend on the actual order of input rows.
## How was this patch tested?
Existing test cases should be enough.
Author: Cheng Lian <lian@databricks.com>
Closes#14749 from liancheng/spark-17182-non-deterministic-collect.
## What changes were proposed in this pull request?
The range operator previously didn't support SQL generation, which made it not possible to use in views.
## How was this patch tested?
Unit tests.
cc hvanhovell
Author: Eric Liang <ekl@databricks.com>
Closes#14724 from ericl/spark-17162.
## What changes were proposed in this pull request?
In 2.0, we change the threshold of splitting expressions from 16K to 64K, which cause very bad performance on wide table, because the generated method can't be JIT compiled by default (above the limit of 8K bytecode).
This PR will decrease it to 1K, based on the benchmark results for a wide table with 400 columns of LongType.
It also fix a bug around splitting expression in whole-stage codegen (it should not split them).
## How was this patch tested?
Added benchmark suite.
Author: Davies Liu <davies@databricks.com>
Closes#14692 from davies/split_exprs.
## What changes were proposed in this pull request?
Currently, `NullPropagation` optimizer replaces `COUNT` on null literals in a bottom-up fashion. During that, `WindowExpression` is not covered properly. This PR adds the missing propagation logic.
**Before**
```scala
scala> sql("SELECT COUNT(1 + NULL) OVER ()").show
java.lang.UnsupportedOperationException: Cannot evaluate expression: cast(0 as bigint) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)
```
**After**
```scala
scala> sql("SELECT COUNT(1 + NULL) OVER ()").show
+----------------------------------------------------------------------------------------------+
|count((1 + CAST(NULL AS INT))) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)|
+----------------------------------------------------------------------------------------------+
| 0|
+----------------------------------------------------------------------------------------------+
```
## How was this patch tested?
Pass the Jenkins test with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14689 from dongjoon-hyun/SPARK-17098.
## What changes were proposed in this pull request?
This patch adds support for SQL generation for inline tables. With this, it would be possible to create a view that depends on inline tables.
## How was this patch tested?
Added a test case in LogicalPlanToSQLSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14709 from petermaxlee/SPARK-17150.
## What changes were proposed in this pull request?
Modifies error message for numeric literals to
Numeric literal <literal> does not fit in range [min, max] for type <T>
## How was this patch tested?
Fixed up the error messages for literals.sql in SqlQueryTestSuite and re-ran via sbt. Also fixed up error messages in ExpressionParserSuite
Author: Srinath Shankar <srinath@databricks.com>
Closes#14721 from srinathshankar/sc4296.
## What changes were proposed in this pull request?
This patch creates array.sql in SQLQueryTestSuite for testing array related functions, including:
- indexing
- array creation
- size
- array_contains
- sort_array
## How was this patch tested?
The patch itself is about adding tests.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14708 from petermaxlee/SPARK-17149.
## What changes were proposed in this pull request?
This patch changes predicate pushdown optimization rule (PushDownPredicate) from using a blacklist to a whitelist. That is to say, operators must be explicitly allowed. This approach is more future-proof: previously it was possible for us to introduce a new operator and then render the optimization rule incorrect.
This also fixes the bug that previously we allowed pushing filter beneath limit, which was incorrect. That is to say, before this patch, the optimizer would rewrite
```
select * from (select * from range(10) limit 5) where id > 3
to
select * from range(10) where id > 3 limit 5
```
## How was this patch tested?
- a unit test case in FilterPushdownSuite
- an end-to-end test in limit.sql
Author: Reynold Xin <rxin@databricks.com>
Closes#14713 from rxin/SPARK-16994.
## What changes were proposed in this pull request?
This patch improves inline table support with the following:
1. Support type coercion.
2. Support using foldable expressions. Previously only literals were supported.
3. Improve error message handling.
4. Improve test coverage.
## How was this patch tested?
Added a new unit test suite ResolveInlineTablesSuite and a new file-based end-to-end test inline-table.sql.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14676 from petermaxlee/SPARK-16947.
## What changes were proposed in this pull request?
This patch fixes the problem described in SPARK-17117, i.e. "SELECT 1 / NULL" throws an analysis exception:
```
org.apache.spark.sql.AnalysisException: cannot resolve '(1 / NULL)' due to data type mismatch: differing types in '(1 / NULL)' (int and null).
```
The problem is that division type coercion did not take null type into account.
## How was this patch tested?
A unit test for the type coercion, and a few end-to-end test cases using SQLQueryTestSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14695 from petermaxlee/SPARK-17117.
## What changes were proposed in this pull request?
This adds analyzer rules for resolving table-valued functions, and adds one builtin implementation for range(). The arguments for range() are the same as those of `spark.range()`.
## How was this patch tested?
Unit tests.
cc hvanhovell
Author: Eric Liang <ekl@databricks.com>
Closes#14656 from ericl/sc-4309.
## What changes were proposed in this pull request?
The `Optimizer` rules `PushThroughSetOperations` and `PushDownPredicate` have a redundant rule to push down `Filter` through `Union`. We should remove it.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#14687 from viirya/remove-extra-pushdown.
## What changes were proposed in this pull request?
I was looking at the code for UnresolvedOrdinal and made a few small changes to make it slightly more clear:
1. Rename the rule to SubstituteUnresolvedOrdinals which is more consistent with other rules that start with verbs. Note that this is still inconsistent with CTESubstitution and WindowsSubstitution.
2. Broke the test suite down from a single test case to three test cases.
## How was this patch tested?
This is a minor cleanup.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14672 from petermaxlee/SPARK-17034.
## What changes were proposed in this pull request?
A TreeNodeException is thrown when executing the following minimal example in Spark 2.0.
import spark.implicits._
case class test (x: Int, q: Int)
val d = Seq(1).toDF("x")
d.withColumn("q", lit(0)).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show
d.withColumn("q", expr("0")).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show
The problem is at `FoldablePropagation`. The rule will do `transformExpressions` on `LogicalPlan`. The query above contains a `MapGroups` which has a parameter `dataAttributes:Seq[Attribute]`. One attributes in `dataAttributes` will be transformed to an `Alias(literal(0), _)` in `FoldablePropagation`. `Alias` is not an `Attribute` and causes the error.
We can't easily detect such type inconsistency during transforming expressions. A direct approach to this problem is to skip doing `FoldablePropagation` on object operators as they should not contain such expressions.
## How was this patch tested?
Jenkins tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#14648 from viirya/flat-mapping.
## What changes were proposed in this pull request?
The current subquery expression interface contains a little bit of technical debt in the form of a few different access paths to get and set the query contained by the expression. This is confusing to anyone who goes over this code.
This PR unifies these access paths.
## How was this patch tested?
(Existing tests)
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14685 from hvanhovell/SPARK-17106.
## What changes were proposed in this pull request?
This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method.
Here is [the original PR](https://github.com/apache/spark/pull/13243) for SPARK-15285. However, it breaks a build with Scala 2.10 since Scala 2.10 does not a case class with large number of members. Thus, it was reverted by [this commit](fa244e5a90).
## How was this patch tested?
Added new tests by using `DefinedByConstructorParams` instead of case class for scala-2.10
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#14670 from kiszk/SPARK-15285-2.
## What changes were proposed in this pull request?
Currently methods in `ParserUtils` are tested indirectly, we should add test cases in `ParserUtilsSuite` to verify their integrity directly.
## How was this patch tested?
New test cases in `ParserUtilsSuite`
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#14620 from jiangxb1987/parserUtils.
## What changes were proposed in this pull request?
This PR adds a field to subquery alias in order to make the usage of views in a resolved `LogicalPlan` more visible (and more understandable).
For example, the following view and query:
```sql
create view constants as select 1 as id union all select 1 union all select 42
select * from constants;
```
...now yields the following analyzed plan:
```
Project [id#39]
+- SubqueryAlias c, `default`.`constants`
+- Project [gen_attr_0#36 AS id#39]
+- SubqueryAlias gen_subquery_0
+- Union
:- Union
: :- Project [1 AS gen_attr_0#36]
: : +- OneRowRelation$
: +- Project [1 AS gen_attr_1#37]
: +- OneRowRelation$
+- Project [42 AS gen_attr_2#38]
+- OneRowRelation$
```
## How was this patch tested?
Added tests for the two code paths in `SessionCatalogSuite` (sql/core) and `HiveMetastoreCatalogSuite` (sql/hive)
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14657 from hvanhovell/SPARK-17068.
## What changes were proposed in this pull request?
This PR renames `ParserUtils.assert` to `ParserUtils.validate`. This is done because this method is used to check requirements, and not to check if the program is in an invalid state.
## How was this patch tested?
Simple rename. Compilation should do.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14665 from hvanhovell/SPARK-17084.
## What changes were proposed in this pull request?
This PR adds expression `UnresolvedOrdinal` to represent the ordinal in GROUP BY or ORDER BY, and fixes the rules when resolving ordinals.
Ordinals in GROUP BY or ORDER BY like `1` in `order by 1` or `group by 1` should be considered as unresolved before analysis. But in current code, it uses `Literal` expression to store the ordinal. This is inappropriate as `Literal` itself is a resolved expression, it gives the user a wrong message that the ordinals has already been resolved.
### Before this change
Ordinal is stored as `Literal` expression
```
scala> sc.setLogLevel("TRACE")
scala> sql("select a from t group by 1 order by 1")
...
'Sort [1 ASC], true
+- 'Aggregate [1], ['a]
+- 'UnresolvedRelation `t
```
For query:
```
scala> Seq(1).toDF("a").createOrReplaceTempView("t")
scala> sql("select count(a), a from t group by 2 having a > 0").show
```
During analysis, the intermediate plan before applying rule `ResolveAggregateFunctions` is:
```
'Filter ('a > 0)
+- Aggregate [2], [count(1) AS count(1)#83L, a#81]
+- LocalRelation [value#7 AS a#9]
```
Before this PR, rule `ResolveAggregateFunctions` believes all expressions of `Aggregate` have already been resolved, and tries to resolve the expressions in `Filter` directly. But this is wrong, as ordinal `2` in Aggregate is not really resolved!
### After this change
Ordinals are stored as `UnresolvedOrdinal`.
```
scala> sc.setLogLevel("TRACE")
scala> sql("select a from t group by 1 order by 1")
...
'Sort [unresolvedordinal(1) ASC], true
+- 'Aggregate [unresolvedordinal(1)], ['a]
+- 'UnresolvedRelation `t`
```
## How was this patch tested?
Unit tests.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14616 from clockfly/spark-16955.
## What changes were proposed in this pull request?
This PR changes the CTE resolving rule to use only **forward-declared** tables in order to prevent infinite loops. More specifically, new logic is like the following.
* Resolve CTEs in `WITH` clauses first before replacing the main SQL body.
* When resolving CTEs, only forward-declared CTEs or base tables are referenced.
- Self-referencing is not allowed any more.
- Cross-referencing is not allowed any more.
**Reported Error Scenarios**
```scala
scala> sql("WITH t AS (SELECT 1 FROM t) SELECT * FROM t")
java.lang.StackOverflowError
...
scala> sql("WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1, t2")
java.lang.StackOverflowError
...
```
Note that `t`, `t1`, and `t2` are not declared in database. Spark falls into infinite loops before resolving table names.
## How was this patch tested?
Pass the Jenkins tests with new two testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14397 from dongjoon-hyun/SPARK-16771-TREENODE.
#### What changes were proposed in this pull request?
So far, the test cases of `TableIdentifierParserSuite` do not cover the quoted cases. We should add one for avoiding regression.
#### How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14244 from gatorsmile/quotedIdentifiers.
## What changes were proposed in this pull request?
This patch updates the SQL parser to parse negative numeric literals as numeric literals, instead of unary minus of positive literals.
This allows the parser to parse the minimal value for each data type, e.g. "-32768S".
## How was this patch tested?
Updated test cases.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14608 from petermaxlee/SPARK-17013.
## What changes were proposed in this pull request?
There could be multiple subqueries that generate same results, we could re-use the result instead of running it multiple times.
This PR also cleanup up how we run subqueries.
For SQL query
```sql
select id,(select avg(id) from t) from t where id > (select avg(id) from t)
```
The explain is
```
== Physical Plan ==
*Project [id#15L, Subquery subquery29 AS scalarsubquery()#35]
: +- Subquery subquery29
: +- *HashAggregate(keys=[], functions=[avg(id#15L)])
: +- Exchange SinglePartition
: +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)])
: +- *Range (0, 1000, splits=4)
+- *Filter (cast(id#15L as double) > Subquery subquery29)
: +- Subquery subquery29
: +- *HashAggregate(keys=[], functions=[avg(id#15L)])
: +- Exchange SinglePartition
: +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)])
: +- *Range (0, 1000, splits=4)
+- *Range (0, 1000, splits=4)
```
The visualized plan:
![reuse-subquery](https://cloud.githubusercontent.com/assets/40902/17573229/e578d93c-5f0d-11e6-8a3c-0150d81d3aed.png)
## How was this patch tested?
Existing tests.
Author: Davies Liu <davies@databricks.com>
Closes#14548 from davies/subq.
## What changes were proposed in this pull request?
This patch adds three test files:
1. arithmetic.sql.out
2. order-by-ordinal.sql
3. group-by-ordinal.sql
This includes https://github.com/apache/spark/pull/14594.
## How was this patch tested?
This is a test case change.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14595 from petermaxlee/SPARK-17015.
## What changes were proposed in this pull request?
This PR adds `MINUS` set operator which is equivalent `EXCEPT DISTINCT`. This will slightly improve the compatibility with Oracle.
## How was this patch tested?
Pass the Jenkins with newly added testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14570 from dongjoon-hyun/SPARK-10601.
## What changes were proposed in this pull request?
Fixed small typo - "value ... ~~in~~ is null"
## How was this patch tested?
Still compiles!
Author: Michał Kiełbowicz <jupblb@users.noreply.github.com>
Closes#14569 from jupblb/typo-fix.
## What changes were proposed in this pull request?
MSCK REPAIR TABLE could be used to recover the partitions in external catalog based on partitions in file system.
Another syntax is: ALTER TABLE table RECOVER PARTITIONS
The implementation in this PR will only list partitions (not the files with a partition) in driver (in parallel if needed).
## How was this patch tested?
Added unit tests for it and Hive compatibility test suite.
Author: Davies Liu <davies@databricks.com>
Closes#14500 from davies/repair_table.
## What changes were proposed in this pull request?
This PR adds argument type information for typed logical plan like MapElements, TypedFilter, and AppendColumn, so that we can use these info in customized optimizer rule.
## How was this patch tested?
Existing test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14494 from clockfly/add_more_info_for_typed_operator.
## What changes were proposed in this pull request?
Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability.
## How was this patch tested?
Existing tests.
Author: Holden Karau <holden@us.ibm.com>
Closes#14407 from holdenk/SPARK-16779.
### What changes were proposed in this pull request?
Currently, the `refreshTable` API is always case sensitive.
When users use the view name without the exact case match, the API silently ignores the call. Users might expect the command has been successfully completed. However, when users run the subsequent SQL commands, they might still get the exception, like
```
Job aborted due to stage failure:
Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 7, localhost):
java.io.FileNotFoundException:
File file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-bd4b9ea6-9aec-49c5-8f05-01cff426211e/part-r-00000-0c84b915-c032-4f2e-abf5-1d48fdbddf38.snappy.parquet does not exist
```
This PR is to fix the issue.
### How was this patch tested?
Added a test case.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14523 from gatorsmile/refreshTempTable.
## What changes were proposed in this pull request?
This patch fixes the incorrect results in the rule ResolveSubquery in Catalyst's Analysis phase by returning an error message when the LIMIT is found in the path from the parent table to the correlated predicate in the subquery.
## How was this patch tested?
./dev/run-tests
a new unit test on the problematic pattern.
Author: Nattavut Sutyanyong <nsy.can@gmail.com>
Closes#14411 from nsyca/master.
## What changes were proposed in this pull request?
This PR is to fix the minor Java linter errors as following:
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/VariableLengthRowBasedKeyValueBatch.java:[42,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/VariableLengthRowBasedKeyValueBatch.java:[97,10] (modifier) RedundantModifier: Redundant 'final' modifier.
## How was this patch tested?
Manual test.
dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
Author: Weiqing Yang <yangweiqing001@gmail.com>
Closes#14532 from Sherry302/master.
## What changes were proposed in this pull request?
regexp_extract actually returns null when it shouldn't when a regex matches but the requested optional group did not. This makes it return an empty string, as apparently designed.
## How was this patch tested?
Additional unit test
Author: Sean Owen <sowen@cloudera.com>
Closes#14504 from srowen/SPARK-16409.
## What changes were proposed in this pull request?
The java.net.URL class has a globally synchronized Hashtable, which limits the throughput of any single executor doing lots of calls to parse_url(). Tests have shown that a 36-core machine can only get to 10% CPU use because the threads are locked most of the time.
This patch switches to java.net.URI which has less features than java.net.URL but focuses on URI parsing, which is enough for parse_url().
New tests were added to make sure a few common edge cases didn't change behaviour.
https://issues.apache.org/jira/browse/SPARK-16826
## How was this patch tested?
I've kept the old URL code commented for now, so that people can verify that the new unit tests do pass with java.net.URL.
Thanks to srowen for the help!
Author: Sylvain Zimmer <sylvain@sylvainzimmer.com>
Closes#14488 from sylvinus/master.
## What changes were proposed in this pull request?
we have various logical plans for CREATE TABLE and CTAS: `CreateTableUsing`, `CreateTableUsingAsSelect`, `CreateHiveTableAsSelectLogicalPlan`. This PR unifies them to reduce the complexity and centralize the error handling.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14482 from cloud-fan/table.
## What changes were proposed in this pull request?
For DataSet typed select:
```
def select[U1: Encoder](c1: TypedColumn[T, U1]): Dataset[U1]
```
If type T is a case class or a tuple class that is not atomic, the resulting logical plan's schema will mismatch with `Dataset[T]` encoder's schema, which will cause encoder error and throw AnalysisException.
### Before change:
```
scala> case class A(a: Int, b: Int)
scala> Seq((0, A(1,2))).toDS.select($"_2".as[A])
org.apache.spark.sql.AnalysisException: cannot resolve '`a`' given input columns: [_2];
..
```
### After change:
```
scala> case class A(a: Int, b: Int)
scala> Seq((0, A(1,2))).toDS.select($"_2".as[A]).show
+---+---+
| a| b|
+---+---+
| 1| 2|
+---+---+
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14474 from clockfly/SPARK-16853.
## What changes were proposed in this pull request?
These 2 methods take `CatalogTable` as parameter, which already have the database information.
## How was this patch tested?
existing test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14476 from cloud-fan/minor5.
## What changes were proposed in this pull request?
Implements `eval()` method for expression `AssertNotNull` so that we can convert local projection on LocalRelation to another LocalRelation.
### Before change:
```
scala> import org.apache.spark.sql.catalyst.dsl.expressions._
scala> import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull
scala> import org.apache.spark.sql.Column
scala> case class A(a: Int)
scala> Seq((A(1),2)).toDS().select(new Column(AssertNotNull("_1".attr, Nil))).explain
java.lang.UnsupportedOperationException: Only code-generated evaluation is supported.
at org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull.eval(objects.scala:850)
...
```
### After the change:
```
scala> Seq((A(1),2)).toDS().select(new Column(AssertNotNull("_1".attr, Nil))).explain(true)
== Parsed Logical Plan ==
'Project [assertnotnull('_1) AS assertnotnull(_1)#5]
+- LocalRelation [_1#2, _2#3]
== Analyzed Logical Plan ==
assertnotnull(_1): struct<a:int>
Project [assertnotnull(_1#2) AS assertnotnull(_1)#5]
+- LocalRelation [_1#2, _2#3]
== Optimized Logical Plan ==
LocalRelation [assertnotnull(_1)#5]
== Physical Plan ==
LocalTableScan [assertnotnull(_1)#5]
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#14486 from clockfly/assertnotnull_eval.
## What changes were proposed in this pull request?
Partition discovery is rather expensive, so we should do it at execution time instead of during physical planning. Right now there is not much benefit since ListingFileCatalog will read scan for all partitions at planning time anyways, but this can be optimized in the future. Also, there might be more information for partition pruning not available at planning time.
This PR moves a lot of the file scan logic from planning to execution time. All file scan operations are handled by `FileSourceScanExec`, which handles both batched and non-batched file scans. This requires some duplication with `RowDataSourceScanExec`, but is probably worth it so that `FileSourceScanExec` does not need to depend on an input RDD.
TODO: In another pr, move DataSourceScanExec to it's own file.
## How was this patch tested?
Existing tests (it might be worth adding a test that catalog.listFiles() is delayed until execution, but this can be delayed until there is an actual benefit to doing so).
Author: Eric Liang <ekl@databricks.com>
Closes#14241 from ericl/refactor.
## What changes were proposed in this pull request?
Here is a table about the behaviours of `array`/`map` and `greatest`/`least` in Hive, MySQL and Postgres:
| |Hive|MySQL|Postgres|
|---|---|---|---|---|
|`array`/`map`|can find a wider type with decimal type arguments, and will truncate the wider decimal type if necessary|can find a wider type with decimal type arguments, no truncation problem|can find a wider type with decimal type arguments, no truncation problem|
|`greatest`/`least`|can find a wider type with decimal type arguments, and truncate if necessary, but can't do string promotion|can find a wider type with decimal type arguments, no truncation problem, but can't do string promotion|can find a wider type with decimal type arguments, no truncation problem, but can't do string promotion|
I think these behaviours makes sense and Spark SQL should follow them.
This PR fixes `array` and `map` by using `findWiderCommonType` to get the wider type.
This PR fixes `greatest` and `least` by add a `findWiderTypeWithoutStringPromotion`, which provides similar semantic of `findWiderCommonType`, but without string promotion.
## How was this patch tested?
new tests in `TypeCoersionSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#14439 from cloud-fan/bug.
## What changes were proposed in this pull request?
`Greatest` and `Least` are not conditional expressions, but arithmetic expressions.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14460 from cloud-fan/move.
## What changes were proposed in this pull request?
In Spark 1.6 (with Hive support) we could use `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions as literals (without adding braces), for example:
```SQL
select /* Spark 1.6: */ current_date, /* Spark 1.6 & Spark 2.0: */ current_date()
```
This was accidentally dropped in Spark 2.0. This PR reinstates this functionality.
## How was this patch tested?
Added a case to ExpressionParserSuite.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#14442 from hvanhovell/SPARK-16836.
## What changes were proposed in this pull request?
There are two related bugs of Python-only UDTs. Because the test case of second one needs the first fix too. I put them into one PR. If it is not appropriate, please let me know.
### First bug: When MapObjects works on Python-only UDTs
`RowEncoder` will use `PythonUserDefinedType.sqlType` for its deserializer expression. If the sql type is `ArrayType`, we will have `MapObjects` working on it. But `MapObjects` doesn't consider `PythonUserDefinedType` as its input data type. It causes error like:
import pyspark.sql.group
from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
from pyspark.sql.types import *
schema = StructType().add("key", LongType()).add("val", PythonOnlyUDT())
df = spark.createDataFrame([(i % 3, PythonOnlyPoint(float(i), float(i))) for i in range(10)], schema=schema)
df.show()
File "/home/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o36.showString.
: java.lang.RuntimeException: Error while decoding: scala.MatchError: org.apache.spark.sql.types.PythonUserDefinedTypef4ceede8 (of class org.apache.spark.sql.types.PythonUserDefinedType)
...
### Second bug: When Python-only UDTs is the element type of ArrayType
import pyspark.sql.group
from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
from pyspark.sql.types import *
schema = StructType().add("key", LongType()).add("val", ArrayType(PythonOnlyUDT()))
df = spark.createDataFrame([(i % 3, [PythonOnlyPoint(float(i), float(i))]) for i in range(10)], schema=schema)
df.show()
## How was this patch tested?
PySpark's sql tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#13778 from viirya/fix-pyudt.
## What changes were proposed in this pull request?
Fix of incorrect arguments (dropping slideDuration and using windowDuration) in constructors for TimeWindow.
The JIRA this addresses is here: https://issues.apache.org/jira/browse/SPARK-16837
## How was this patch tested?
Added a test to TimeWindowSuite to check that the results of TimeWindow object apply and TimeWindow class constructor are equivalent.
Author: Tom Magrino <tmagrino@fb.com>
Closes#14441 from tmagrino/windowing-fix.
## What changes were proposed in this pull request?
Greatest/least function does not have the most friendly error message for data types. This patch improves the error message to not show the Seq type, and use more human readable data types.
Before:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST (ArrayBuffer(DecimalType(2,1), StringType)).; line 1 pos 7
```
After:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST(decimal(2,1), string).; line 1 pos 7
```
## How was this patch tested?
Manually verified the output and also added unit tests to ConditionalExpressionSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14453 from petermaxlee/SPARK-16850.
## What changes were proposed in this pull request?
These 2 expressions are not needed anymore after we have `Greatest` and `Least`. This PR removes them and related tests.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14434 from cloud-fan/minor1.
## What changes were proposed in this pull request?
Removes the deprecated timestamp constructor and incidentally fixes the use which was using system timezone rather than the one specified when working near DST.
This change also causes the roundtrip tests to fail since it now actually uses all the timezones near DST boundaries where it didn't before.
Note: this is only a partial the solution, longer term we should follow up with https://issues.apache.org/jira/browse/SPARK-16788 to avoid this problem & simplify our timezone handling code.
## How was this patch tested?
New tests for two timezones added so even if user timezone happens to coincided with one, the other tests should still fail. Important note: this (temporarily) disables the round trip tests until we can fix the issue more thoroughly.
Author: Holden Karau <holden@us.ibm.com>
Closes#14398 from holdenk/SPARK-16774-fix-use-of-deprecated-timestamp-constructor.
## What changes were proposed in this pull request?
a failing test case + fix to SPARK-16791 (https://issues.apache.org/jira/browse/SPARK-16791)
## How was this patch tested?
added a failing test case to CastSuit, then fixed the Cast code and rerun the entire CastSuit
Author: eyal farago <eyal farago>
Author: Eyal Farago <eyal.farago@actimize.com>
Closes#14400 from eyalfa/SPARK-16791_cast_struct_with_timestamp_field_fails.
## What changes were proposed in this pull request?
Currently, `UNION` queries on incompatible types show misleading error messages, i.e., `unresolved operator Union`. We had better show a more correct message. This will help users in the situation of [SPARK-16704](https://issues.apache.org/jira/browse/SPARK-16704).
**Before**
```scala
scala> sql("select 1,2,3 union (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Union;
scala> sql("select 1,2,3 intersect (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Intersect;
scala> sql("select 1,2,3 except (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: unresolved operator 'Except;
```
**After**
```scala
scala> sql("select 1,2,3 union (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the second column of the second table;
scala> sql("select 1,2,3 intersect (select 1,array(2),3)")
org.apache.spark.sql.AnalysisException: Intersect can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the second column of the second table;
scala> sql("select 1,2,3 except (select array(1),array(2),3)")
org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the first column of the second table;
```
## How was this patch tested?
Pass the Jenkins test with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14355 from dongjoon-hyun/SPARK-16726.
## What changes were proposed in this pull request?
`StructField` has very similar semantic with `CatalogColumn`, except that `CatalogColumn` use string to express data type. I think it's reasonable to use `StructType` as the `CatalogTable.schema` and remove `CatalogColumn`.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14363 from cloud-fan/column.
## What changes were proposed in this pull request?
The catalyst package is meant to be internal, and as a result it does not make sense to mark things as private[sql] or private[spark]. It simply makes debugging harder when Spark developers need to inspect the plans at runtime.
This patch removes all private[sql] and private[spark] visibility modifiers in org.apache.spark.sql.catalyst.
## How was this patch tested?
N/A - just visibility changes.
Author: Reynold Xin <rxin@databricks.com>
Closes#14418 from rxin/SPARK-16813.
## What changes were proposed in this pull request?
Use foreach/for instead of map where operation requires execution of body, not actually defining a transformation
## How was this patch tested?
Jenkins
Author: Sean Owen <sowen@cloudera.com>
Closes#14332 from srowen/SPARK-16694.
## What changes were proposed in this pull request?
We do not want SparkExceptions from job failures in the planning phase to create TreeNodeException. Hence do not wrap SparkException in TreeNodeException.
## How was this patch tested?
New unit test
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#14395 from tdas/SPARK-16748.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.
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.
## 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.
## 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.
## 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.
## 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 17eec0a71b (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.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.
## 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.
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.
## 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.
## What changes were proposed in this pull request?
This patch moves regexp related unit tests from StringExpressionsSuite to RegexpExpressionsSuite to match the file name for regexp expressions.
## How was this patch tested?
This is a test only change.
Author: Reynold Xin <rxin@databricks.com>
Closes#14230 from rxin/SPARK-16584.
## What changes were proposed in this pull request?
This patch is just a slightly safer way to fix the issue we encountered in https://github.com/apache/spark/pull/14168 should this pattern re-occur at other places in the code.
## How was this patch tested?
Existing tests. Also, I manually tested that it fixes the problem in SPARK-16514 without having the proposed change in https://github.com/apache/spark/pull/14168
Author: Sameer Agarwal <sameerag@cs.berkeley.edu>
Closes#14227 from sameeragarwal/codegen.
#### What changes were proposed in this pull request?
Based on the [Hive SQL syntax](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-ChangeColumnName/Type/Position/Comment), the command to change column name/type/position/comment is `ALTER TABLE CHANGE COLUMN`. However, in our .g4 file, it is `ALTER TABLE CHANGE COLUMNS`. Because it is the last optional keyword, it does not take any effect. Thus, I put the issue as a Trivial level.
cc hvanhovell
#### How was this patch tested?
Existing test cases
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14186 from gatorsmile/changeColumns.
## What changes were proposed in this pull request?
`Alias` with metadata is not a no-op and we should not strip it in `RemoveAliasOnlyProject` rule.
This PR also did some improvement for this rule:
1. extend the semantic of `alias-only`. Now we allow the project list to be partially aliased.
2. add unit test for this rule.
## How was this patch tested?
new `RemoveAliasOnlyProjectSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14106 from cloud-fan/bug.
## What changes were proposed in this pull request?
Currently our Optimizer may reorder the predicates to run them more efficient, but in non-deterministic condition, change the order between deterministic parts and non-deterministic parts may change the number of input rows. For example:
```SELECT a FROM t WHERE rand() < 0.1 AND a = 1```
And
```SELECT a FROM t WHERE a = 1 AND rand() < 0.1```
may call rand() for different times and therefore the output rows differ.
This PR improved this condition by checking whether the predicate is placed before any non-deterministic predicates.
## How was this patch tested?
Expanded related testcases in FilterPushdownSuite.
Author: 蒋星博 <jiangxingbo@meituan.com>
Closes#14012 from jiangxb1987/ppd.
## What changes were proposed in this pull request?
RegexExtract and RegexReplace currently crash on non-nullable input due use of a hard-coded local variable name (e.g. compiles fail with `java.lang.Exception: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 85, Column 26: Redefinition of local variable "m" `).
This changes those variables to use fresh names, and also in a few other places.
## How was this patch tested?
Unit tests. rxin
Author: Eric Liang <ekl@databricks.com>
Closes#14168 from ericl/sc-3906.
## 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.
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.
## 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.
## 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.
## 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.
## 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.
#### 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.
## 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.
## 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.
## 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.
## What changes were proposed in this pull request?
This small patch modifies ExpressionEvalHelper. checkEvaluation to support comparing NaN values for floating point comparisons.
## How was this patch tested?
This is a test harness change.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14103 from petermaxlee/SPARK-16436.
## What changes were proposed in this pull request?
In #13537 we truncate `simpleString` if it is a long `StructType`. But sometimes we need `catalogString` to reconstruct `TypeInfo`, for example in description of [SPARK-16415 ](https://issues.apache.org/jira/browse/SPARK-16415). So we need to keep the implementation of `catalogString` not affected by our truncate.
## How was this patch tested?
added a test case.
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes#14089 from adrian-wang/catalogstring.
## What changes were proposed in this pull request?
This PR improves `OptimizeIn` optimizer to remove the literal repetitions from SQL `IN` predicates. This optimizer prevents user mistakes and also can optimize some queries like [TPCDS-36](https://github.com/apache/spark/blob/master/sql/core/src/test/resources/tpcds/q36.sql#L19).
**Before**
```scala
scala> sql("select state from (select explode(array('CA','TN')) state) where state in ('TN','TN','TN','TN','TN','TN','TN')").explain
== Physical Plan ==
*Filter state#6 IN (TN,TN,TN,TN,TN,TN,TN)
+- Generate explode([CA,TN]), false, false, [state#6]
+- Scan OneRowRelation[]
```
**After**
```scala
scala> sql("select state from (select explode(array('CA','TN')) state) where state in ('TN','TN','TN','TN','TN','TN','TN')").explain
== Physical Plan ==
*Filter state#6 IN (TN)
+- Generate explode([CA,TN]), false, false, [state#6]
+- Scan OneRowRelation[]
```
## How was this patch tested?
Pass the Jenkins tests (including a new testcase).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13876 from dongjoon-hyun/SPARK-16174.
## 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.
#### What changes were proposed in this pull request?
Different from the other leaf nodes, `MetastoreRelation` and `SimpleCatalogRelation` have a pre-defined `alias`, which is used to change the qualifier of the node. However, based on the existing alias handling, alias should be put in `SubqueryAlias`.
This PR is to separate alias handling from `MetastoreRelation` and `SimpleCatalogRelation` to make it consistent with the other nodes. It simplifies the signature and conversion to a `BaseRelation`.
For example, below is an example query for `MetastoreRelation`, which is converted to a `LogicalRelation`:
```SQL
SELECT tmp.a + 1 FROM test_parquet_ctas tmp WHERE tmp.a > 2
```
Before changes, the analyzed plan is
```
== Analyzed Logical Plan ==
(a + 1): int
Project [(a#951 + 1) AS (a + 1)#952]
+- Filter (a#951 > 2)
+- SubqueryAlias tmp
+- Relation[a#951] parquet
```
After changes, the analyzed plan becomes
```
== Analyzed Logical Plan ==
(a + 1): int
Project [(a#951 + 1) AS (a + 1)#952]
+- Filter (a#951 > 2)
+- SubqueryAlias tmp
+- SubqueryAlias test_parquet_ctas
+- Relation[a#951] parquet
```
**Note: the optimized plans are the same.**
For `SimpleCatalogRelation`, the existing code always generates two Subqueries. Thus, no change is needed.
#### How was this patch tested?
Added test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14053 from gatorsmile/removeAliasFromMetastoreRelation.
## 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 24bea00047
## How was this patch tested?
Unit test in `MetastoreDataSourcesSuite.scala`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#13517 from HyukjinKwon/SPARK-14839.
## 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.
## What changes were proposed in this pull request?
This PR implements `stack` table generating function.
## How was this patch tested?
Pass the Jenkins tests including new testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14033 from dongjoon-hyun/SPARK-16286.
## What changes were proposed in this pull request?
This patch fixes the bug that the refresh command does not work on temporary views. This patch is based on https://github.com/apache/spark/pull/13989, but removes the public Dataset.refresh() API as well as improved test coverage.
Note that I actually think the public refresh() API is very useful. We can in the future implement it by also invalidating the lazy vals in QueryExecution (or alternatively just create a new QueryExecution).
## How was this patch tested?
Re-enabled a previously ignored test, and added a new test suite for Hive testing behavior of temporary views against MetastoreRelation.
Author: Reynold Xin <rxin@databricks.com>
Author: petermaxlee <petermaxlee@gmail.com>
Closes#14009 from rxin/SPARK-16311.
## What changes were proposed in this pull request?
This PR implements `inline` table generating function.
## How was this patch tested?
Pass the Jenkins tests with new testcase.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13976 from dongjoon-hyun/SPARK-16288.
## What changes were proposed in this pull request?
This PR adds `map_keys` and `map_values` SQL functions in order to remove Hive fallback.
## How was this patch tested?
Pass the Jenkins tests including new testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13967 from dongjoon-hyun/SPARK-16278.
#### What changes were proposed in this pull request?
Star expansion over a table containing zero column does not work since 1.6. However, it works in Spark 1.5.1. This PR is to fix the issue in the master branch.
For example,
```scala
val rddNoCols = sqlContext.sparkContext.parallelize(1 to 10).map(_ => Row.empty)
val dfNoCols = sqlContext.createDataFrame(rddNoCols, StructType(Seq.empty))
dfNoCols.registerTempTable("temp_table_no_cols")
sqlContext.sql("select * from temp_table_no_cols").show
```
Without the fix, users will get the following the exception:
```
java.lang.IllegalArgumentException: requirement failed
at scala.Predef$.require(Predef.scala:221)
at org.apache.spark.sql.catalyst.analysis.UnresolvedStar.expand(unresolved.scala:199)
```
#### How was this patch tested?
Tests are added
Author: gatorsmile <gatorsmile@gmail.com>
Closes#14007 from gatorsmile/starExpansionTableWithZeroColumn.
## What changes were proposed in this pull request?
This PR fixes the minor Java linter errors like the following.
```
- public int read(char cbuf[], int off, int len) throws IOException {
+ public int read(char[] cbuf, int off, int len) throws IOException {
```
## How was this patch tested?
Manual.
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14017 from dongjoon-hyun/minor_build_java_linter_error.
## What changes were proposed in this pull request?
This PR adds a new logical optimizer, `PropagateEmptyRelation`, to collapse a logical plans consisting of only empty LocalRelations.
**Optimizer Targets**
1. Binary(or Higher)-node Logical Plans
- Union with all empty children.
- Join with one or two empty children (including Intersect/Except).
2. Unary-node Logical Plans
- Project/Filter/Sample/Join/Limit/Repartition with all empty children.
- Aggregate with all empty children and without AggregateFunction expressions, COUNT.
- Generate with Explode because other UserDefinedGenerators like Hive UDTF returns results.
**Sample Query**
```sql
WITH t1 AS (SELECT a FROM VALUES 1 t(a)),
t2 AS (SELECT b FROM VALUES 1 t(b) WHERE 1=2)
SELECT a,b
FROM t1, t2
WHERE a=b
GROUP BY a,b
HAVING a>1
ORDER BY a,b
```
**Before**
```scala
scala> sql("with t1 as (select a from values 1 t(a)), t2 as (select b from values 1 t(b) where 1=2) select a,b from t1, t2 where a=b group by a,b having a>1 order by a,b").explain
== Physical Plan ==
*Sort [a#0 ASC, b#1 ASC], true, 0
+- Exchange rangepartitioning(a#0 ASC, b#1 ASC, 200)
+- *HashAggregate(keys=[a#0, b#1], functions=[])
+- Exchange hashpartitioning(a#0, b#1, 200)
+- *HashAggregate(keys=[a#0, b#1], functions=[])
+- *BroadcastHashJoin [a#0], [b#1], Inner, BuildRight
:- *Filter (isnotnull(a#0) && (a#0 > 1))
: +- LocalTableScan [a#0]
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
+- *Filter (isnotnull(b#1) && (b#1 > 1))
+- LocalTableScan <empty>, [b#1]
```
**After**
```scala
scala> sql("with t1 as (select a from values 1 t(a)), t2 as (select b from values 1 t(b) where 1=2) select a,b from t1, t2 where a=b group by a,b having a>1 order by a,b").explain
== Physical Plan ==
LocalTableScan <empty>, [a#0, b#1]
```
## How was this patch tested?
Pass the Jenkins tests (including a new testsuite).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13906 from dongjoon-hyun/SPARK-16208.
## What changes were proposed in this pull request?
During the code generation, a `LocalRelation` often has a huge `Vector` object as `data`. In the simple example below, a `LocalRelation` has a Vector with 1000000 elements of `UnsafeRow`.
```
val numRows = 1000000
val ds = (1 to numRows).toDS().persist()
benchmark.addCase("filter+reduce") { iter =>
ds.filter(a => (a & 1) == 0).reduce(_ + _)
}
```
At `TreeNode.transformChildren`, all elements of the vector is unnecessarily iterated to check whether any children exist in the vector since `Vector` is Traversable. This part significantly increases code generation time.
This patch avoids this overhead by checking the number of children before iterating all elements; `LocalRelation` does not have children since it extends `LeafNode`.
The performance of the above example
```
without this patch
Java HotSpot(TM) 64-Bit Server VM 1.8.0_91-b14 on Mac OS X 10.11.5
Intel(R) Core(TM) i5-5257U CPU 2.70GHz
compilationTime: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
filter+reduce 4426 / 4533 0.2 4426.0 1.0X
with this patch
compilationTime: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
filter+reduce 3117 / 3391 0.3 3116.6 1.0X
```
## How was this patch tested?
using existing unit tests
Author: Hiroshi Inoue <inouehrs@jp.ibm.com>
Closes#14000 from inouehrs/compilation-time-reduction.
## What changes were proposed in this pull request?
This patch implements the elt function, as it is implemented in Hive.
## How was this patch tested?
Added expression unit test in StringExpressionsSuite and end-to-end test in StringFunctionsSuite.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#13966 from petermaxlee/SPARK-16276.
## What changes were proposed in this pull request?
This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive.
**Before**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7
```
**After**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
+---+---+-----+
|pos|key|value|
+---+---+-----+
| 0| a| 1|
| 1| b| 2|
+---+---+-----+
```
For `array` argument, `after` is the same with `before`.
```
scala> sql("select posexplode(array(1, 2, 3))").show
+---+---+
|pos|col|
+---+---+
| 0| 1|
| 1| 2|
| 2| 3|
+---+---+
```
## How was this patch tested?
Pass the Jenkins tests with newly added testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13971 from dongjoon-hyun/SPARK-16289.
## What changes were proposed in this pull request?
Force the sorter to Spill when number of elements in the pointer array reach a certain size. This is to workaround the issue of timSort failing on large buffer size.
## How was this patch tested?
Tested by running a job which was failing without this change due to TimSort bug.
Author: Sital Kedia <skedia@fb.com>
Closes#13107 from sitalkedia/fix_TimSort.
## What changes were proposed in this pull request?
This PR Checks the size limit when doubling the array size in BufferHolder to avoid integer overflow.
## How was this patch tested?
Manual test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13829 from clockfly/SPARK-16071_2.
## What changes were proposed in this pull request?
This patch implements xpath_boolean expression for Spark SQL, a xpath function that returns true or false. The implementation is modelled after Hive's xpath_boolean, except that how the expression handles null inputs. Hive throws a NullPointerException at runtime if either of the input is null. This implementation returns null if either of the input is null.
## How was this patch tested?
Created two new test suites. One for unit tests covering the expression, and the other for end-to-end test in SQL.
Author: petermaxlee <petermaxlee@gmail.com>
Closes#13964 from petermaxlee/SPARK-16274.
## What changes were proposed in this pull request?
This PR adds 3 optimizer rules for typed filter:
1. push typed filter down through `SerializeFromObject` and eliminate the deserialization in filter condition.
2. pull typed filter up through `SerializeFromObject` and eliminate the deserialization in filter condition.
3. combine adjacent typed filters and share the deserialized object among all the condition expressions.
This PR also adds `TypedFilter` logical plan, to separate it from normal filter, so that the concept is more clear and it's easier to write optimizer rules.
## How was this patch tested?
`TypedFilterOptimizationSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13846 from cloud-fan/filter.
## What changes were proposed in this pull request?
This extends SPARK-15860 to include metrics for the actual bytecode size of janino-generated methods. They can be accessed in the same way as any other codahale metric, e.g.
```
scala> org.apache.spark.metrics.source.CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.getSnapshot().getValues()
res7: Array[Long] = Array(532, 532, 532, 542, 1479, 2670, 3585, 3585)
scala> org.apache.spark.metrics.source.CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.getSnapshot().getValues()
res8: Array[Long] = Array(5, 5, 5, 5, 10, 10, 10, 10, 15, 15, 15, 38, 63, 79, 88, 94, 94, 94, 132, 132, 165, 165, 220, 220)
```
## How was this patch tested?
Small unit test, also verified manually that the performance impact is minimal (<10%). hvanhovell
Author: Eric Liang <ekl@databricks.com>
Closes#13934 from ericl/spark-16238.
## What changes were proposed in this pull request?
The analyzer rule for resolving using joins should respect the case sensitivity setting.
## How was this patch tested?
New tests in ResolveNaturalJoinSuite
Author: Yin Huai <yhuai@databricks.com>
Closes#13977 from yhuai/SPARK-16301.
#### What changes were proposed in this pull request?
Based on the previous discussion with cloud-fan hvanhovell in another related PR https://github.com/apache/spark/pull/13764#discussion_r67994276, it looks reasonable to add convenience methods for users to add `comment` when defining `StructField`.
Currently, the column-related `comment` attribute is stored in `Metadata` of `StructField`. For example, users can add the `comment` attribute using the following way:
```Scala
StructType(
StructField(
"cl1",
IntegerType,
nullable = false,
new MetadataBuilder().putString("comment", "test").build()) :: Nil)
```
This PR is to add more user friendly methods for the `comment` attribute when defining a `StructField`. After the changes, users are provided three different ways to do it:
```Scala
val struct = (new StructType)
.add("a", "int", true, "test1")
val struct = (new StructType)
.add("c", StringType, true, "test3")
val struct = (new StructType)
.add(StructField("d", StringType).withComment("test4"))
```
#### How was this patch tested?
Added test cases:
- `DataTypeSuite` is for testing three types of API changes,
- `DataFrameReaderWriterSuite` is for parquet, json and csv formats - using in-memory catalog
- `OrcQuerySuite.scala` is for orc format using Hive-metastore
Author: gatorsmile <gatorsmile@gmail.com>
Closes#13860 from gatorsmile/newMethodForComment.
## What changes were proposed in this pull request?
`MAX(COUNT(*))` is invalid since aggregate expression can't be nested within another aggregate expression. This case should be captured at analysis phase, but somehow sneaks off to runtime.
The reason is that when checking aggregate expressions in `CheckAnalysis`, a checking branch treats all expressions that reference no input attributes as valid ones. However, `MAX(COUNT(*))` is translated into `MAX(COUNT(1))` at analysis phase and also references no input attribute.
This PR fixes this issue by removing the aforementioned branch.
## How was this patch tested?
New test case added in `AnalysisErrorSuite`.
Author: Cheng Lian <lian@databricks.com>
Closes#13968 from liancheng/spark-16291-nested-agg-functions.
## What changes were proposed in this pull request?
This patch ports Hive's UDFXPathUtil over to Spark, which can be used to implement xpath functionality in Spark in the near future.
## How was this patch tested?
Added two new test suites UDFXPathUtilSuite and ReusableStringReaderSuite. They have been ported over from Hive (but rewritten in Scala in order to leverage ScalaTest).
Author: petermaxlee <petermaxlee@gmail.com>
Closes#13961 from petermaxlee/xpath.
## What changes were proposed in this pull request?
This patch removes the blind fallback into Hive for functions. Instead, it creates a whitelist and adds only a small number of functions to the whitelist, i.e. the ones we intend to support in the long run in Spark.
## How was this patch tested?
Updated tests to reflect the change.
Author: Reynold Xin <rxin@databricks.com>
Closes#13939 from rxin/hive-whitelist.
## What changes were proposed in this pull request?
Fixes a couple old references to `DataFrameWriter.startStream` to `DataStreamWriter.start
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#13952 from brkyvz/minor-doc-fix.
## What changes were proposed in this pull request?
The root cause is in `MapObjects`. Its parameter `loopVar` is not declared as child, but sometimes can be same with `lambdaFunction`(e.g. the function that takes `loopVar` and produces `lambdaFunction` may be `identity`), which is a child. This brings trouble when call `withNewChildren`, it may mistakenly treat `loopVar` as a child and cause `IndexOutOfBoundsException: 0` later.
This PR fixes this bug by simply pulling out the paremters from `LambdaVariable` and pass them to `MapObjects` directly.
## How was this patch tested?
new test in `DatasetAggregatorSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13835 from cloud-fan/map-objects.
## What changes were proposed in this pull request?
The root cause is: the output attributes of outer join are derived from its children, while they are actually different attributes(outer join can return null).
We have already added some special logic to handle it, e.g. `PushPredicateThroughJoin` won't push down predicates through outer join side, `FixNullability`.
This PR adds one more special logic in `FoldablePropagation`.
## How was this patch tested?
new test in `DataFrameSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13884 from cloud-fan/bug.
## What changes were proposed in this pull request?
Spark currently shows all functions when issue a `SHOW FUNCTIONS` command. This PR refines the `SHOW FUNCTIONS` command by allowing users to select all functions, user defined function or system functions. The following syntax can be used:
**ALL** (default)
```SHOW FUNCTIONS```
```SHOW ALL FUNCTIONS```
**SYSTEM**
```SHOW SYSTEM FUNCTIONS```
**USER**
```SHOW USER FUNCTIONS```
## How was this patch tested?
Updated tests and added tests to the DDLSuite
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#13929 from hvanhovell/SPARK-16220.
## What changes were proposed in this pull request?
- Fix tests regarding show functions functionality
- Revert `catalog.ListFunctions` and `SHOW FUNCTIONS` to return to `Spark 1.X` functionality.
Cherry picked changes from this PR: https://github.com/apache/spark/pull/13413/files
## How was this patch tested?
Unit tests.
Author: Bill Chambers <bill@databricks.com>
Author: Bill Chambers <wchambers@ischool.berkeley.edu>
Closes#13916 from anabranch/master.
## What changes were proposed in this pull request?
This pr is to remove `hashCode` and `equals` in `ArrayBasedMapData` because the type cannot be used as join keys, grouping keys, or in equality tests.
## How was this patch tested?
Add a new test suite `MapDataSuite` for comparison tests.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#13847 from maropu/UnsafeMapTest.
## What changes were proposed in this pull request?
Currently the initial buffer size in the sorter is hard coded inside the code and is too small for large workload. As a result, the sorter spends significant time expanding the buffer size and copying the data. It would be useful to have it configurable.
## How was this patch tested?
Tested by running a job on the cluster.
Author: Sital Kedia <skedia@fb.com>
Closes#13699 from sitalkedia/config_sort_buffer_upstream.
## What changes were proposed in this pull request?
`CollectSet` cannot have map-typed data because MapTypeData does not implement `equals`.
So, this pr is to add type checks in `CheckAnalysis`.
## How was this patch tested?
Added tests to check failures when we found map-typed data in `CollectSet`.
Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>
Closes#13892 from maropu/SPARK-16192.
## What changes were proposed in this pull request?
Replace use of `commons-lang` in favor of `commons-lang3` and forbid the former via scalastyle; remove `NotImplementedException` from `comons-lang` in favor of JDK `UnsupportedOperationException`
## How was this patch tested?
Jenkins tests
Author: Sean Owen <sowen@cloudera.com>
Closes#13843 from srowen/SPARK-16129.
## What changes were proposed in this pull request?
It's weird that `ParserUtils.operationNotAllowed` returns an exception and the caller throw it.
## How was this patch tested?
N/A
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13874 from cloud-fan/style.
## What changes were proposed in this pull request?
This PR changes `CombineFilters` to compose the final predicate condition by using (`child predicate` AND `parent predicate`) instead of (`parent predicate` AND `child predicate`). This is a best effort approach. Some other optimization rules may destroy this order by reorganizing conjunctive predicates.
**Reported Error Scenario**
Chris McCubbin reported a bug when he used StringIndexer in an ML pipeline with additional filters. It seems that during filter pushdown, we changed the ordering in the logical plan.
```scala
import org.apache.spark.ml.feature._
val df1 = (0 until 3).map(_.toString).toDF
val indexer = new StringIndexer()
.setInputCol("value")
.setOutputCol("idx")
.setHandleInvalid("skip")
.fit(df1)
val df2 = (0 until 5).map(_.toString).toDF
val predictions = indexer.transform(df2)
predictions.show() // this is okay
predictions.where('idx > 2).show() // this will throw an exception
```
Please see the notebook at https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/1233855/2159162931615821/588180/latest.html for error messages.
## How was this patch tested?
Pass the Jenkins tests (including a new testcase).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13872 from dongjoon-hyun/SPARK-16164.
## What changes were proposed in this pull request?
This calculation of statistics is not trivial anymore, it could be very slow on large query (for example, TPC-DS Q64 took several minutes to plan).
During the planning of a query, the statistics of any logical plan should not change (even InMemoryRelation), so we should use `lazy val` to cache the statistics.
For InMemoryRelation, the statistics could be updated after materialization, it's only useful when used in another query (before planning), because once we finished the planning, the statistics will not be used anymore.
## How was this patch tested?
Testsed with TPC-DS Q64, it could be planned in a second after the patch.
Author: Davies Liu <davies@databricks.com>
Closes#13871 from davies/fix_statistics.
## What changes were proposed in this pull request?
Currently, we use local timezone to parse or format a timestamp (TimestampType), then use Long as the microseconds since epoch UTC.
In from_utc_timestamp() and to_utc_timestamp(), we did not consider the local timezone, they could return different results with different local timezone.
This PR will do the conversion based on human time (in local timezone), it should return same result in whatever timezone. But because the mapping from absolute timestamp to human time is not exactly one-to-one mapping, it will still return wrong result in some timezone (also in the begging or ending of DST).
This PR is kind of the best effort fix. In long term, we should make the TimestampType be timezone aware to fix this totally.
## How was this patch tested?
Tested these function in all timezone.
Author: Davies Liu <davies@databricks.com>
Closes#13784 from davies/convert_tz.
## What changes were proposed in this pull request?
Although the top level input object can not be null, but when we use `Encoders.tuple` to combine 2 encoders, their input objects are not top level anymore and can be null. We should handle this case.
## How was this patch tested?
new test in DatasetSuite
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13807 from cloud-fan/bug.
## What changes were proposed in this pull request?
This PR adds the static partition support to INSERT statement when the target table is a data source table.
## How was this patch tested?
New tests in InsertIntoHiveTableSuite and DataSourceAnalysisSuite.
**Note: This PR is based on https://github.com/apache/spark/pull/13766. The last commit is the actual change.**
Author: Yin Huai <yhuai@databricks.com>
Closes#13769 from yhuai/SPARK-16030-1.
## What changes were proposed in this pull request?
Internally, we use Int to represent a date (the days since 1970-01-01), when we convert that into unix timestamp (milli-seconds since epoch in UTC), we get the offset of a timezone using local millis (the milli-seconds since 1970-01-01 in a timezone), but TimeZone.getOffset() expect unix timestamp, the result could be off by one hour (in Daylight Saving Time (DST) or not).
This PR change to use best effort approximate of posix timestamp to lookup the offset. In the event of changing of DST, Some time is not defined (for example, 2016-03-13 02:00:00 PST), or could lead to multiple valid result in UTC (for example, 2016-11-06 01:00:00), this best effort approximate should be enough in practice.
## How was this patch tested?
Added regression tests.
Author: Davies Liu <davies@databricks.com>
Closes#13652 from davies/fix_timezone.
## What changes were proposed in this pull request?
The current table insertion has some weird behaviours:
1. inserting into a partitioned table with mismatch columns has confusing error message for hive table, and wrong result for datasource table
2. inserting into a partitioned table without partition list has wrong result for hive table.
This PR fixes these 2 problems.
## How was this patch tested?
new test in hive `SQLQuerySuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13754 from cloud-fan/insert2.
## What changes were proposed in this pull request?
This small patch renames a few optimizer rules to make the naming more consistent, e.g. class name start with a verb. The main important "fix" is probably SamplePushDown -> PushProjectThroughSample. SamplePushDown is actually the wrong name, since the rule is not about pushing Sample down.
## How was this patch tested?
Updated test cases.
Author: Reynold Xin <rxin@databricks.com>
Closes#13732 from rxin/SPARK-16014.
#### What changes were proposed in this pull request?
`IF NOT EXISTS` in `INSERT OVERWRITE` should not support dynamic partitions. If we specify `IF NOT EXISTS`, the inserted statement is not shown in the table.
This PR is to issue an exception in this case, just like what Hive does. Also issue an exception if users specify `IF NOT EXISTS` if users do not specify any `PARTITION` specification.
#### How was this patch tested?
Added test cases into `PlanParserSuite` and `InsertIntoHiveTableSuite`
Author: gatorsmile <gatorsmile@gmail.com>
Closes#13447 from gatorsmile/insertIfNotExist.
## What changes were proposed in this pull request?
`UTF8String` and all `Unsafe*` classes are backed by either on-heap or off-heap byte arrays. The code generated version `SortMergeJoin` buffers the left hand side join keys during iteration. This was actually problematic in off-heap mode when one of the keys is a `UTF8String` (or any other 'Unsafe*` object) and the left hand side iterator was exhausted (and released its memory); the buffered keys would reference freed memory. This causes Seg-faults and all kinds of other undefined behavior when we would use one these buffered keys.
This PR fixes this problem by creating copies of the buffered variables. I have added a general method to the `CodeGenerator` for this. I have checked all places in which this could happen, and only `SortMergeJoin` had this problem.
This PR is largely based on the work of robbinspg and he should be credited for this.
closes https://github.com/apache/spark/pull/13707
## How was this patch tested?
Manually tested on problematic workloads.
Author: Pete Robbins <robbinspg@gmail.com>
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#13723 from hvanhovell/SPARK-15822-2.
## What changes were proposed in this pull request?
This PR contains a few changes on code comments.
- `HiveTypeCoercion` is renamed into `TypeCoercion`.
- `NoSuchDatabaseException` is only used for the absence of database.
- For partition type inference, only `DoubleType` is considered.
## How was this patch tested?
N/A
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#13674 from dongjoon-hyun/minor_doc_types.
## What changes were proposed in this pull request?
I've found some minor issues in "show tables" command:
1. In the `SessionCatalog.scala`, `listTables(db: String)` method will call `listTables(formatDatabaseName(db), "*")` to list all the tables for certain db, but in the method `listTables(db: String, pattern: String)`, this db name is formatted once more. So I think we should remove
`formatDatabaseName()` in the caller.
2. I suggest to add sort to listTables(db: String) in InMemoryCatalog.scala, just like listDatabases().
## How was this patch tested?
The existing test cases should cover it.
Author: bomeng <bmeng@us.ibm.com>
Closes#13695 from bomeng/SPARK-15978.
#### What changes were proposed in this pull request?
~~If the temp table already exists, we should not silently replace it when doing `CACHE TABLE AS SELECT`. This is inconsistent with the behavior of `CREAT VIEW` or `CREATE TABLE`. This PR is to fix this silent drop.~~
~~Maybe, we also can introduce new syntax for replacing the existing one. For example, in Hive, to replace a view, the syntax should be like `ALTER VIEW AS SELECT` or `CREATE OR REPLACE VIEW AS SELECT`~~
The table name in `CACHE TABLE AS SELECT` should NOT contain database prefix like "database.table". Thus, this PR captures this in Parser and outputs a better error message, instead of reporting the view already exists.
In addition, refactoring the `Parser` to generate table identifiers instead of returning the table name string.
#### How was this patch tested?
- Added a test case for caching and uncaching qualified table names
- Fixed a few test cases that do not drop temp table at the end
- Added the related test case for the issue resolved in this PR
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes#13572 from gatorsmile/cacheTableAsSelect.
## What changes were proposed in this pull request?
gapply() applies an R function on groups grouped by one or more columns of a DataFrame, and returns a DataFrame. It is like GroupedDataSet.flatMapGroups() in the Dataset API.
Please, let me know what do you think and if you have any ideas to improve it.
Thank you!
## How was this patch tested?
Unit tests.
1. Primitive test with different column types
2. Add a boolean column
3. Compute average by a group
Author: Narine Kokhlikyan <narine.kokhlikyan@gmail.com>
Author: NarineK <narine.kokhlikyan@us.ibm.com>
Closes#12836 from NarineK/gapply2.
## What changes were proposed in this pull request?
This PR fixes the problem that Divide Expression inside Aggregation function is casted to wrong type, which cause `select 1/2` and `select sum(1/2)`returning different result.
**Before the change:**
```
scala> sql("select 1/2 as a").show()
+---+
| a|
+---+
|0.5|
+---+
scala> sql("select sum(1/2) as a").show()
+---+
| a|
+---+
|0 |
+---+
scala> sql("select sum(1 / 2) as a").schema
res4: org.apache.spark.sql.types.StructType = StructType(StructField(a,LongType,true))
```
**After the change:**
```
scala> sql("select 1/2 as a").show()
+---+
| a|
+---+
|0.5|
+---+
scala> sql("select sum(1/2) as a").show()
+---+
| a|
+---+
|0.5|
+---+
scala> sql("select sum(1/2) as a").schema
res4: org.apache.spark.sql.types.StructType = StructType(StructField(a,DoubleType,true))
```
## How was this patch tested?
Unit test.
This PR is based on https://github.com/apache/spark/pull/13524 by Sephiroth-Lin
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13651 from clockfly/SPARK-15776.
## What changes were proposed in this pull request?
Two issues I've found for "show databases" command:
1. The returned database name list was not sorted, it only works when "like" was used together; (HIVE will always return a sorted list)
2. When it is used as sql("show databases").show, it will output a table with column named as "result", but for sql("show tables").show, it will output the column name as "tableName", so I think we should be consistent and use "databaseName" at least.
## How was this patch tested?
Updated existing test case to test its ordering as well.
Author: bomeng <bmeng@us.ibm.com>
Closes#13671 from bomeng/SPARK-15952.
## What changes were proposed in this pull request?
`DataFrame` with plan overriding `sameResult` but not using canonicalized plan to compare can't cacheTable.
The example is like:
```
val localRelation = Seq(1, 2, 3).toDF()
localRelation.createOrReplaceTempView("localRelation")
spark.catalog.cacheTable("localRelation")
assert(
localRelation.queryExecution.withCachedData.collect {
case i: InMemoryRelation => i
}.size == 1)
```
and this will fail as:
```
ArrayBuffer() had size 0 instead of expected size 1
```
The reason is that when do `spark.catalog.cacheTable("localRelation")`, `CacheManager` tries to cache for the plan wrapped by `SubqueryAlias` but when planning for the DataFrame `localRelation`, `CacheManager` tries to find cached table for the not-wrapped plan because the plan for DataFrame `localRelation` is not wrapped.
Some plans like `LocalRelation`, `LogicalRDD`, etc. override `sameResult` method, but not use canonicalized plan to compare so the `CacheManager` can't detect the plans are the same.
This pr modifies them to use canonicalized plan when override `sameResult` method.
## How was this patch tested?
Added a test to check if DataFrame with plan overriding sameResult but not using canonicalized plan to compare can cacheTable.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#13638 from ueshin/issues/SPARK-15915.
## What changes were proposed in this pull request?
In our encoder framework, we imply that serializer expressions should use `BoundReference` to refer to the input object, and a lot of codes depend on this contract(e.g. ExpressionEncoder.tuple). This PR adds some document and assert in `ExpressionEncoder` to make it clearer.
## How was this patch tested?
existing tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13648 from cloud-fan/comment.
## What changes were proposed in this pull request?
SparkSession.catalog.listFunctions currently returns all functions, including the list of built-in functions. This makes the method not as useful because anytime it is run the result set contains over 100 built-in functions.
## How was this patch tested?
CatalogSuite
Author: Sandeep Singh <sandeep@techaddict.me>
Closes#13413 from techaddict/SPARK-15663.
## What changes were proposed in this pull request?
This PR enforces schema check when converting DataFrame to Dataset using Kryo encoder. For example.
**Before the change:**
Schema is NOT checked when converting DataFrame to Dataset using kryo encoder.
```
scala> case class B(b: Int)
scala> implicit val encoder = Encoders.kryo[B]
scala> val df = Seq((1)).toDF("b")
scala> val ds = df.as[B] // Schema compatibility is NOT checked
```
**After the change:**
Report AnalysisException since the schema is NOT compatible.
```
scala> val ds = Seq((1)).toDF("b").as[B]
org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(`b` AS BINARY)' due to data type mismatch: cannot cast IntegerType to BinaryType;
...
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13632 from clockfly/spark-15910.
# What changes were proposed in this pull request?
This pull request fixes the COUNT bug in the `RewriteCorrelatedScalarSubquery` rule.
After this change, the rule tests the expression at the root of the correlated subquery to determine whether the expression returns `NULL` on empty input. If the expression does not return `NULL`, the rule generates additional logic in the `Project` operator above the rewritten subquery. This additional logic intercepts `NULL` values coming from the outer join and replaces them with the value that the subquery's expression would return on empty input.
This PR takes over https://github.com/apache/spark/pull/13155. It only fixes an issue with `Literal` construction and style issues. All credits should go frreiss.
# How was this patch tested?
Added regression tests to cover all branches of the updated rule (see changes to `SubquerySuite`).
Ran all existing automated regression tests after merging with latest trunk.
Author: frreiss <frreiss@us.ibm.com>
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#13629 from hvanhovell/SPARK-15370-cleanup.
## What changes were proposed in this pull request?
Queries with embedded existential sub-query predicates throws exception when building the physical plan.
Example failing query:
```SQL
scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1")
scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2")
scala> sql("select c1 from t1 where (case when c2 in (select c2 from t2) then 2 else 3 end) IN (select c2 from t1)").show()
Binding attribute, tree: c2#239
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: c2#239
at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50)
at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88)
...
at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:87)
at org.apache.spark.sql.execution.joins.HashJoin$$anonfun$4.apply(HashJoin.scala:66)
at org.apache.spark.sql.execution.joins.HashJoin$$anonfun$4.apply(HashJoin.scala:66)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.immutable.List.foreach(List.scala:381)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.immutable.List.map(List.scala:285)
at org.apache.spark.sql.execution.joins.HashJoin$class.org$apache$spark$sql$execution$joins$HashJoin$$x$8(HashJoin.scala:66)
at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.org$apache$spark$sql$execution$joins$HashJoin$$x$8$lzycompute(BroadcastHashJoinExec.scala:38)
at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.org$apache$spark$sql$execution$joins$HashJoin$$x$8(BroadcastHashJoinExec.scala:38)
at org.apache.spark.sql.execution.joins.HashJoin$class.buildKeys(HashJoin.scala:63)
at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.buildKeys$lzycompute(BroadcastHashJoinExec.scala:38)
at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.buildKeys(BroadcastHashJoinExec.scala:38)
at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.requiredChildDistribution(BroadcastHashJoinExec.scala:52)
```
**Problem description:**
When the left hand side expression of an existential sub-query predicate contains another embedded sub-query predicate, the RewritePredicateSubquery optimizer rule does not resolve the embedded sub-query expressions into existential joins.For example, the above query has the following optimized plan, which fails during physical plan build.
```SQL
== Optimized Logical Plan ==
Project [_1#224 AS c1#227]
+- Join LeftSemi, (CASE WHEN predicate-subquery#255 [(_2#225 = c2#239)] THEN 2 ELSE 3 END = c2#228#262)
: +- SubqueryAlias predicate-subquery#255 [(_2#225 = c2#239)]
: +- LocalRelation [c2#239]
:- LocalRelation [_1#224, _2#225]
+- LocalRelation [c2#228#262]
== Physical Plan ==
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: c2#239
```
**Solution:**
In RewritePredicateSubquery, before rewriting the outermost predicate sub-query, resolve any embedded existential sub-queries. The Optimized plan for the above query after the changes looks like below.
```SQL
== Optimized Logical Plan ==
Project [_1#224 AS c1#227]
+- Join LeftSemi, (CASE WHEN exists#285 THEN 2 ELSE 3 END = c2#228#284)
:- Join ExistenceJoin(exists#285), (_2#225 = c2#239)
: :- LocalRelation [_1#224, _2#225]
: +- LocalRelation [c2#239]
+- LocalRelation [c2#228#284]
== Physical Plan ==
*Project [_1#224 AS c1#227]
+- *BroadcastHashJoin [CASE WHEN exists#285 THEN 2 ELSE 3 END], [c2#228#284], LeftSemi, BuildRight
:- *BroadcastHashJoin [_2#225], [c2#239], ExistenceJoin(exists#285), BuildRight
: :- LocalTableScan [_1#224, _2#225]
: +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
: +- LocalTableScan [c2#239]
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
+- LocalTableScan [c2#228#284]
+- LocalTableScan [c222#36], [[111],[222]]
```
## How was this patch tested?
Added new test cases in SubquerySuite.scala
Author: Ioana Delaney <ioanamdelaney@gmail.com>
Closes#13570 from ioana-delaney/fixEmbedSubPredV1.
## What changes were proposed in this pull request?
This pull request fixes the COUNT bug in the `RewriteCorrelatedScalarSubquery` rule.
After this change, the rule tests the expression at the root of the correlated subquery to determine whether the expression returns NULL on empty input. If the expression does not return NULL, the rule generates additional logic in the Project operator above the rewritten subquery. This additional logic intercepts NULL values coming from the outer join and replaces them with the value that the subquery's expression would return on empty input.
## How was this patch tested?
Added regression tests to cover all branches of the updated rule (see changes to `SubquerySuite.scala`).
Ran all existing automated regression tests after merging with latest trunk.
Author: frreiss <frreiss@us.ibm.com>
Closes#13155 from frreiss/master.
## What changes were proposed in this pull request?
Adds codahale metrics for the codegen source text size and how long it takes to compile. The size is particularly interesting, since the JVM does have hard limits on how large methods can get.
To simplify, I added the metrics under a statically-initialized source that is always registered with SparkEnv.
## How was this patch tested?
Unit tests
Author: Eric Liang <ekl@databricks.com>
Closes#13586 from ericl/spark-15860.
## What changes were proposed in this pull request?
This adds support for radix sort of nullable long fields. When a sort field is null and radix sort is enabled, we keep nulls in a separate region of the sort buffer so that radix sort does not need to deal with them. This also has performance benefits when sorting smaller integer types, since the current representation of nulls in two's complement (Long.MIN_VALUE) otherwise forces a full-width radix sort.
This strategy for nulls does mean the sort is no longer stable. cc davies
## How was this patch tested?
Existing randomized sort tests for correctness. I also tested some TPCDS queries and there does not seem to be any significant regression for non-null sorts.
Some test queries (best of 5 runs each).
Before change:
scala> val start = System.nanoTime; spark.range(5000000).selectExpr("if(id > 5, cast(hash(id) as long), NULL) as h").coalesce(1).orderBy("h").collect(); (System.nanoTime - start) / 1e6
start: Long = 3190437233227987
res3: Double = 4716.471091
After change:
scala> val start = System.nanoTime; spark.range(5000000).selectExpr("if(id > 5, cast(hash(id) as long), NULL) as h").coalesce(1).orderBy("h").collect(); (System.nanoTime - start) / 1e6
start: Long = 3190367870952791
res4: Double = 2981.143045
Author: Eric Liang <ekl@databricks.com>
Closes#13161 from ericl/sc-2998.
## What changes were proposed in this pull request?
Spark currently incorrectly continues to use cached data even if the underlying data is overwritten.
Current behavior:
```scala
val dir = "/tmp/test"
sqlContext.range(1000).write.mode("overwrite").parquet(dir)
val df = sqlContext.read.parquet(dir).cache()
df.count() // outputs 1000
sqlContext.range(10).write.mode("overwrite").parquet(dir)
sqlContext.read.parquet(dir).count() // outputs 1000 <---- We are still using the cached dataset
```
This patch fixes this bug by adding support for `REFRESH path` that invalidates and refreshes all the cached data (and the associated metadata) for any dataframe that contains the given data source path.
Expected behavior:
```scala
val dir = "/tmp/test"
sqlContext.range(1000).write.mode("overwrite").parquet(dir)
val df = sqlContext.read.parquet(dir).cache()
df.count() // outputs 1000
sqlContext.range(10).write.mode("overwrite").parquet(dir)
spark.catalog.refreshResource(dir)
sqlContext.read.parquet(dir).count() // outputs 10 <---- We are not using the cached dataset
```
## How was this patch tested?
Unit tests for overwrites and appends in `ParquetQuerySuite` and `CachedTableSuite`.
Author: Sameer Agarwal <sameer@databricks.com>
Closes#13566 from sameeragarwal/refresh-path-2.
## What changes were proposed in this pull request?
The base class `SpecificParquetRecordReaderBase` used for vectorized parquet reader will try to get pushed-down filters from the given configuration. This pushed-down filters are used for RowGroups-level filtering. However, we don't set up the filters to push down into the configuration. In other words, the filters are not actually pushed down to do RowGroups-level filtering. This patch is to fix this and tries to set up the filters for pushing down to configuration for the reader.
## How was this patch tested?
Existing tests should be passed.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#13371 from viirya/vectorized-reader-push-down-filter.
## What changes were proposed in this pull request?
As discussed in https://github.com/apache/spark/pull/12836
we need to override stringArgs method in MapPartitionsInR in order to avoid too large strings generated by "stringArgs" method based on the input arguments.
In this case exclude some of the input arguments: serialized R objects.
## How was this patch tested?
Existing test cases
Author: Narine Kokhlikyan <narine.kokhlikyan@gmail.com>
Closes#13610 from NarineK/dapply_MapPartitionsInR_stringArgs.
## What changes were proposed in this pull request?
Serializer instantiation will consider existing SparkConf
## How was this patch tested?
manual test with `ImmutableList` (Guava) and `kryo-serializers`'s `Immutable*Serializer` implementations.
Added Test Suite.
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: Sela <ansela@paypal.com>
Closes#13424 from amitsela/SPARK-15489.
## What changes were proposed in this pull request?
Code generated `SortMergeJoin` failed with wrong results when using structs as keys. This could (eventually) be traced back to the use of a wrong row reference when comparing structs.
## How was this patch tested?
TBD
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#13589 from hvanhovell/SPARK-15822.
## What changes were proposed in this pull request?
In scala, immutable.List.length is an expensive operation so we should
avoid using Seq.length == 0 or Seq.lenth > 0, and use Seq.isEmpty and Seq.nonEmpty instead.
## How was this patch tested?
existing tests
Author: wangyang <wangyang@haizhi.com>
Closes#13601 from yangw1234/isEmpty.
## What changes were proposed in this pull request?
Replace all occurrences of `None: Option[X]` with `Option.empty[X]`
## How was this patch tested?
Exisiting Tests
Author: Sandeep Singh <sandeep@techaddict.me>
Closes#13591 from techaddict/minor-7.
## What changes were proposed in this pull request?
This PR moves `QueryPlanner.planLater()` method into `GenericStrategy` for extra strategies to be able to use `planLater` in its strategy.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <ueshin@happy-camper.st>
Closes#13147 from ueshin/issues/SPARK-6320.
## What changes were proposed in this pull request?
This patch moves some codes in `DataFrameWriter.insertInto` that belongs to `Analyzer`.
## How was this patch tested?
Existing tests.
Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Closes#13496 from viirya/move-analyzer-stuff.
## What changes were proposed in this pull request?
When the output mode is complete, then the output of a streaming aggregation essentially will contain the complete aggregates every time. So this is not different from a batch dataset within an incremental execution. Other non-streaming operations should be supported on this dataset. In this PR, I am just adding support for sorting, as it is a common useful functionality. Support for other operations will come later.
## How was this patch tested?
Additional unit tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#13549 from tdas/SPARK-15812.
## What changes were proposed in this pull request?
With very wide tables, e.g. thousands of fields, the plan output is unreadable and often causes OOMs due to inefficient string processing. This truncates all struct and operator field lists to a user configurable threshold to limit performance impact.
It would also be nice to optimize string generation to avoid these sort of O(n^2) slowdowns entirely (i.e. use StringBuilder everywhere including expressions), but this is probably too large of a change for 2.0 at this point, and truncation has other benefits for usability.
## How was this patch tested?
Added a microbenchmark that covers this case particularly well. I also ran the microbenchmark while varying the truncation threshold.
```
numFields = 5
wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem) 2336 / 2558 0.0 23364.4 0.1X
numFields = 25
wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem) 4237 / 4465 0.0 42367.9 0.1X
numFields = 100
wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem) 10458 / 11223 0.0 104582.0 0.0X
numFields = Infinity
wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------
[info] java.lang.OutOfMemoryError: Java heap space
```
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>
Closes#13537 from ericl/truncated-string.
## What changes were proposed in this pull request?
The current implementations of `UnixTime` and `FromUnixTime` do not cache their parser/formatter as much as they could. This PR resolved this issue.
This PR is a take over from https://github.com/apache/spark/pull/13522 and further optimizes the re-use of the parser/formatter. It also fixes the improves handling (catching the actual exception instead of `Throwable`). All credits for this work should go to rajeshbalamohan.
This PR closes https://github.com/apache/spark/pull/13522
## How was this patch tested?
Current tests.
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Rajesh Balamohan <rbalamohan@apache.org>
Closes#13581 from hvanhovell/SPARK-14321.
## What changes were proposed in this pull request?
The help function 'toStructType' in the AttributeSeq class doesn't include the metadata when it builds the StructField, so it causes this reported problem https://issues.apache.org/jira/browse/SPARK-15804?jql=project%20%3D%20SPARK when spark writes the the dataframe with the metadata to the parquet datasource.
The code path is when spark writes the dataframe to the parquet datasource through the InsertIntoHadoopFsRelationCommand, spark will build the WriteRelation container, and it will call the help function 'toStructType' to create StructType which contains StructField, it should include the metadata there, otherwise, we will lost the user provide metadata.
## How was this patch tested?
added test case in ParquetQuerySuite.scala
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: Kevin Yu <qyu@us.ibm.com>
Closes#13555 from kevinyu98/spark-15804.
## What changes were proposed in this pull request?
The parser currently does not allow the use of some SQL keywords as table or field names. This PR adds supports for all keywords as identifier. The exception to this are table aliases, in this case most keywords are allowed except for join keywords (```anti, full, inner, left, semi, right, natural, on, join, cross```) and set-operator keywords (```union, intersect, except```).
## How was this patch tested?
I have added/move/renamed test in the catalyst `*ParserSuite`s.
Author: Herman van Hovell <hvanhovell@databricks.com>
Closes#13534 from hvanhovell/SPARK-15789.
## What changes were proposed in this pull request?
The current implementation of "CREATE TEMPORARY TABLE USING datasource..." is NOT creating any intermediate temporary data directory like temporary HDFS folder, instead, it only stores a SQL string in memory. Probably we should use "TEMPORARY VIEW" instead.
This PR assumes a temporary table has to link with some temporary intermediate data. It follows the definition of temporary table like this (from [hortonworks doc](https://docs.hortonworks.com/HDPDocuments/HDP2/HDP-2.3.0/bk_dataintegration/content/temp-tables.html)):
> A temporary table is a convenient way for an application to automatically manage intermediate data generated during a complex query
**Example**:
```
scala> spark.sql("CREATE temporary view my_tab7 (c1: String, c2: String) USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat OPTIONS (PATH '/Users/seanzhong/csv/cars.csv')")
scala> spark.sql("select c1, c2 from my_tab7").show()
+----+-----+
| c1| c2|
+----+-----+
|year| make|
|2012|Tesla|
...
```
It NOW prints a **deprecation warning** if "CREATE TEMPORARY TABLE USING..." is used.
```
scala> spark.sql("CREATE temporary table my_tab7 (c1: String, c2: String) USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat OPTIONS (PATH '/Users/seanzhong/csv/cars.csv')")
16/05/31 10:39:27 WARN SparkStrategies$DDLStrategy: CREATE TEMPORARY TABLE tableName USING... is deprecated, please use CREATE TEMPORARY VIEW viewName USING... instead
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13414 from clockfly/create_temp_view_using.
## What changes were proposed in this pull request?
This PR allows customization of verbosity in explain output. After change, `dataframe.explain()` and `dataframe.explain(true)` has different verbosity output for physical plan.
Currently, this PR only enables verbosity string for operator `HashAggregateExec` and `SortAggregateExec`. We will gradually enable verbosity string for more operators in future.
**Less verbose mode:** dataframe.explain(extended = false)
`output=[count(a)#85L]` is **NOT** displayed for HashAggregate.
```
scala> Seq((1,2,3)).toDF("a", "b", "c").createTempView("df2")
scala> spark.sql("select count(a) from df2").explain()
== Physical Plan ==
*HashAggregate(key=[], functions=[count(1)])
+- Exchange SinglePartition
+- *HashAggregate(key=[], functions=[partial_count(1)])
+- LocalTableScan
```
**Verbose mode:** dataframe.explain(extended = true)
`output=[count(a)#85L]` is displayed for HashAggregate.
```
scala> spark.sql("select count(a) from df2").explain(true) // "output=[count(a)#85L]" is added
...
== Physical Plan ==
*HashAggregate(key=[], functions=[count(1)], output=[count(a)#85L])
+- Exchange SinglePartition
+- *HashAggregate(key=[], functions=[partial_count(1)], output=[count#87L])
+- LocalTableScan
```
## How was this patch tested?
Manual test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13535 from clockfly/verbose_breakdown_2.
## What changes were proposed in this pull request?
This PR makes sure the typed Filter doesn't change the Dataset schema.
**Before the change:**
```
scala> val df = spark.range(0,9)
scala> df.schema
res12: org.apache.spark.sql.types.StructType = StructType(StructField(id,LongType,false))
scala> val afterFilter = df.filter(_=>true)
scala> afterFilter.schema // !!! schema is CHANGED!!! Column name is changed from id to value, nullable is changed from false to true.
res13: org.apache.spark.sql.types.StructType = StructType(StructField(value,LongType,true))
```
SerializeFromObject and DeserializeToObject are inserted to wrap the Filter, and these two can possibly change the schema of Dataset.
**After the change:**
```
scala> afterFilter.schema // schema is NOT changed.
res47: org.apache.spark.sql.types.StructType = StructType(StructField(id,LongType,false))
```
## How was this patch tested?
Unit test.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#13529 from clockfly/spark-15632.
BindReferences contains a n^2 loop which causes performance issues when operating over large schemas: to determine the ordinal of an attribute reference, we perform a linear scan over the `input` array. Because input can sometimes be a `List`, the call to `input(ordinal).nullable` can also be O(n).
Instead of performing a linear scan, we can convert the input into an array and build a hash map to map from expression ids to ordinals. The greater up-front cost of the map construction is offset by the fact that an expression can contain multiple attribute references, so the cost of the map construction is amortized across a number of lookups.
Perf. benchmarks to follow. /cc ericl
Author: Josh Rosen <joshrosen@databricks.com>
Closes#13505 from JoshRosen/bind-references-improvement.
## What changes were proposed in this pull request?
`an -> a`
Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} && echo {}"` to generate candidates, and review them one by one.
## How was this patch tested?
manual tests
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Closes#13515 from zhengruifeng/an_a.
## What changes were proposed in this pull request?
This PR improves the error handling of `RowEncoder`. When we create a `RowEncoder` with a given schema, we should validate the data type of input object. e.g. we should throw an exception when a field is boolean but is declared as a string column.
This PR also removes the support to use `Product` as a valid external type of struct type. This support is added at https://github.com/apache/spark/pull/9712, but is incomplete, e.g. nested product, product in array are both not working. However, we never officially support this feature and I think it's ok to ban it.
## How was this patch tested?
new tests in `RowEncoderSuite`.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13401 from cloud-fan/bug.
## What changes were proposed in this pull request?
In forType function of object RandomDataGenerator, the code following:
if (maybeSqlTypeGenerator.isDefined){
....
Some(generator)
} else{
None
}
will be changed. Instead, maybeSqlTypeGenerator.map will be used.
## How was this patch tested?
All of the current unit tests passed.
Author: Weiqing Yang <yangweiqing001@gmail.com>
Closes#13448 from Sherry302/master.
We should cache `Metadata.hashCode` and use a singleton for `Metadata.empty` because calculating metadata hashCodes appears to be a bottleneck for certain workloads.
We should also cache `StructType.hashCode`.
In an optimizer stress-test benchmark run by ericl, these `hashCode` calls accounted for roughly 40% of the total CPU time and this bottleneck was completely eliminated by the caching added by this patch.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#13504 from JoshRosen/metadata-fix.
## What changes were proposed in this pull request?
For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL doesn't allow row to be null, only its columns can be null.
This PR explicitly add this constraint and throw exception if users break it.
## How was this patch tested?
several new tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#13469 from cloud-fan/null-object.