### What changes were proposed in this pull request?
Make a special case in `ResolveReferences`, which resolves `OverwriteByExpression`'s condition expression based on the table relation instead of the input query.
### Why are the changes needed?
The condition expression is passed to the table implementation at the end, so we should resolve it using table schema. Previously it works because we have a hack in `ResolveReferences` to delay the resolution if `outputResolved == false`. However, this hack doesn't work for tables accepting any schema like https://github.com/delta-io/delta/pull/521 . We may wrongly resolve the delete condition using input query's outout columns which don't match the table column names.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
existing tests and updated test in v2 write.
Closes#30318 from cloud-fan/v2-write.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR intends to fix the behaviour of query filters in `TPCDSQueryBenchmark`. We can use an option `--query-filter` for selecting TPCDS queries to run, e.g., `--query-filter q6,q8,q13`. But, the current master has a weird behaviour about the option. For example, if we pass `--query-filter q6` so as to run the TPCDS q6 only, `TPCDSQueryBenchmark` runs `q6` and `q6-v2.7` because the `filterQueries` method does not respect the name suffix. So, there is no way now to run the TPCDS q6 only.
### Why are the changes needed?
Bugfix.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Manually checked.
Closes#30324 from maropu/FilterBugInTPCDSQueryBenchmark.
Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
This PR proposes to migrate `SHOW CREATE TABLE` to use `UnresolvedTableOrView` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).
Note that `SHOW CREATE TABLE` works only with a v1 table and a permanent view, and not supported for v2 tables.
### Why are the changes needed?
The changes allow consistent resolution behavior when resolving the table identifier. For example, the following is the current behavior:
```scala
sql("CREATE TEMPORARY VIEW t AS SELECT 1")
sql("CREATE DATABASE db")
sql("CREATE TABLE t (key INT, value STRING) USING hive")
sql("USE db")
sql("SHOW CREATE TABLE t AS SERDE") // Succeeds
```
With this change, `SHOW CREATE TABLE ... AS SERDE` above fails with the following:
```
org.apache.spark.sql.AnalysisException: t is a temp view not table or permanent view.; line 1 pos 0
at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews$$anonfun$apply$7.$anonfun$applyOrElse$43(Analyzer.scala:883)
at scala.Option.map(Option.scala:230)
```
, which is expected since temporary view is resolved first and `SHOW CREATE TABLE ... AS SERDE` doesn't support a temporary view.
Note that there is no behavior change for `SHOW CREATE TABLE` without `AS SERDE` since it was already resolving to a temporary view first. See below for more detail.
### Does this PR introduce _any_ user-facing change?
After this PR, `SHOW CREATE TABLE t AS SERDE` is resolved to a temp view `t` instead of table `db.t` in the above scenario.
Note that there is no behavior change for `SHOW CREATE TABLE` without `AS SERDE`, but the exception message changes from `SHOW CREATE TABLE is not supported on a temporary view` to `t is a temp view not table or permanent view`.
### How was this patch tested?
Updated existing tests.
Closes#30321 from imback82/show_create_table.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
In the PR, I propose to gather common `SHOW TABLES` tests into one trait `org.apache.spark.sql.execution.command.ShowTablesSuite`, and put datasource specific tests to the `v1.ShowTablesSuite` and `v2.ShowTablesSuite`. Also tests for parsing `SHOW TABLES` are extracted to `ShowTablesParserSuite`.
### Why are the changes needed?
- The unification will allow to run common `SHOW TABLES` tests for both DSv1 and DSv2
- We can detect missing features and differences between DSv1 and DSv2 implementations.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
By running new test suites:
- `org.apache.spark.sql.execution.command.v1.ShowTablesSuite`
- `org.apache.spark.sql.execution.command.v2.ShowTablesSuite`
- `ShowTablesParserSuite`
Closes#30287 from MaxGekk/unify-dsv1_v2-tests.
Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Make Literal support char array.
### Why are the changes needed?
We always use `Literal()` to create foldable value, and `char[]` is a usual data type. We can make it easy that support create String Literal with `char[]`.
### Does this PR introduce _any_ user-facing change?
Yes, user can call `Literal()` with `char[]`.
### How was this patch tested?
Add test.
Closes#30295 from ulysses-you/SPARK-33390.
Authored-by: ulysses <youxiduo@weidian.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
The following query produces incorrect results:
```
SELECT date_trunc('minute', '1769-10-17 17:10:02')
```
Spark currently incorrectly returns
```
1769-10-17 17:10:02
```
against the expected return value of
```
1769-10-17 17:10:00
```
**Steps to repro**
Run the following commands in spark-shell:
```
spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
spark.sql("SELECT date_trunc('minute', '1769-10-17 17:10:02')").show()
```
This happens as `truncTimestamp` in package `org.apache.spark.sql.catalyst.util.DateTimeUtils` incorrectly assumes that time zone offsets can never have the granularity of a second and thus does not account for time zone adjustment when truncating the given timestamp to `minute`.
This assumption is currently used when truncating the timestamps to `microsecond, millisecond, second, or minute`.
This PR fixes this issue and always uses time zone knowledge when truncating timestamps regardless of the truncation unit.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added new tests to `DateTimeUtilsSuite` which previously failed and pass now.
Closes#30303 from utkarsh39/trunc-timestamp-fix.
Authored-by: Utkarsh <utkarsh.agarwal@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
### What changes were proposed in this pull request?
Currently we skip subexpression elimination in branches of conditional expressions including `If`, `CaseWhen`, and `Coalesce`. Actually we can do subexpression elimination for such branches if the subexpression is common across all branches. This patch proposes to support subexpression elimination in branches of conditional expressions.
### Why are the changes needed?
We may miss subexpression elimination chances in branches of conditional expressions. This kind of subexpression is frequently seen. It may be written manually by users or come from query optimizer. For example, project collapsing could embed expressions between two `Project`s and produces conditional expression like:
```
CASE WHEN jsonToStruct(json).a = '1' THEN 1.0 WHEN jsonToStruct(json).a = '2' THEN 2.0 ... ELSE 1.2 END
```
If `jsonToStruct(json)` is time-expensive expression, we don't eliminate the duplication and waste time on running it repeatedly now.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test.
Closes#30245 from viirya/SPARK-33337.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
Changes
pyspark.sql.dataframe.DataFrame
to
:py:class:`pyspark.sql.DataFrame`
### Why are the changes needed?
Consistency (see https://github.com/apache/spark/pull/30285#pullrequestreview-526764104).
### Does this PR introduce _any_ user-facing change?
User will see shorter reference with a link.
### How was this patch tested?
`dev/lint-python` and manual check of the rendered docs.
Closes#30313 from zero323/SPARK-33251-FOLLOW-UP.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: Huaxin Gao <huaxing@us.ibm.com>
### What changes were proposed in this pull request?
This removes the `sharesHadoopClasses` flag from `IsolatedClientLoader` in Hive module.
### Why are the changes needed?
Currently, when initializing `IsolatedClientLoader`, users can set the `sharesHadoopClasses` flag to decide whether the `HiveClient` created should share Hadoop classes with Spark itself or not. In the latter case, the client will only load Hadoop classes from the Hive dependencies.
There are two reasons to remove this:
1. this feature is currently used in two cases: 1) unit tests, 2) when the Hadoop version defined in Maven can not be found when `spark.sql.hive.metastore.jars` is equal to "maven", which could be very rare.
2. when `sharesHadoopClasses` is false, Spark doesn't really only use Hadoop classes from Hive jars: we also download `hadoop-client` jar and put all the sub-module jars (e.g., `hadoop-common`, `hadoop-hdfs`) together with the Hive jars, and the Hadoop version used by `hadoop-client` is the same version used by Spark itself. As result, we're mixing two versions of Hadoop jars in the classpath, which could potentially cause issues, especially considering that the default Hadoop version is already 3.2.0 while most Hive versions supported by the `IsolatedClientLoader` is still using Hadoop 2.x or even lower.
### Does this PR introduce _any_ user-facing change?
This affects Spark users in one scenario: when `spark.sql.hive.metastore.jars` is set to `maven` AND the Hadoop version specified in pom file cannot be downloaded, currently the behavior is to switch to _not_ share Hadoop classes, but with the PR it will share Hadoop classes with Spark.
### How was this patch tested?
Existing UTs.
Closes#30284 from sunchao/SPARK-33376.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Push down filter through expand. For case below:
```
create table t1(pid int, uid int, sid int, dt date, suid int) using parquet;
create table t2(pid int, vs int, uid int, csid int) using parquet;
SELECT
years,
appversion,
SUM(uusers) AS users
FROM (SELECT
Date_trunc('year', dt) AS years,
CASE
WHEN h.pid = 3 THEN 'iOS'
WHEN h.pid = 4 THEN 'Android'
ELSE 'Other'
END AS viewport,
h.vs AS appversion,
Count(DISTINCT u.uid) AS uusers
,Count(DISTINCT u.suid) AS srcusers
FROM t1 u
join t2 h
ON h.uid = u.uid
GROUP BY 1,
2,
3) AS a
WHERE viewport = 'iOS'
GROUP BY 1,
2
```
Plan. before this pr:
```
== Physical Plan ==
*(5) HashAggregate(keys=[years#30, appversion#32], functions=[sum(uusers#33L)])
+- Exchange hashpartitioning(years#30, appversion#32, 200), true, [id=#251]
+- *(4) HashAggregate(keys=[years#30, appversion#32], functions=[partial_sum(uusers#33L)])
+- *(4) HashAggregate(keys=[date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12], functions=[count(if ((gid#44 = 1)) u.`uid`#47 else null)])
+- Exchange hashpartitioning(date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, 200), true, [id=#246]
+- *(3) HashAggregate(keys=[date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12], functions=[partial_count(if ((gid#44 = 1)) u.`uid`#47 else null)])
+- *(3) HashAggregate(keys=[date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, u.`uid`#47, u.`suid`#48, gid#44], functions=[])
+- Exchange hashpartitioning(date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, u.`uid`#47, u.`suid`#48, gid#44, 200), true, [id=#241]
+- *(2) HashAggregate(keys=[date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, u.`uid`#47, u.`suid`#48, gid#44], functions=[])
+- *(2) Filter (CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46 = iOS)
+- *(2) Expand [ArrayBuffer(date_trunc(year, cast(dt#9 as timestamp), Some(Etc/GMT+7)), CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END, vs#12, uid#7, null, 1), ArrayBuffer(date_trunc(year, cast(dt#9 as timestamp), Some(Etc/GMT+7)), CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END, vs#12, null, suid#10, 2)], [date_trunc('year', CAST(u.`dt` AS TIMESTAMP))#45, CASE WHEN (h.`pid` = 3) THEN 'iOS' WHEN (h.`pid` = 4) THEN 'Android' ELSE 'Other' END#46, vs#12, u.`uid`#47, u.`suid`#48, gid#44]
+- *(2) Project [uid#7, dt#9, suid#10, pid#11, vs#12]
+- *(2) BroadcastHashJoin [uid#7], [uid#13], Inner, BuildRight
:- *(2) Project [uid#7, dt#9, suid#10]
: +- *(2) Filter isnotnull(uid#7)
: +- *(2) ColumnarToRow
: +- FileScan parquet default.t1[uid#7,dt#9,suid#10] Batched: true, DataFilters: [isnotnull(uid#7)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-bin-hadoop3.2/spark-warehouse/t1], PartitionFilters: [], PushedFilters: [IsNotNull(uid)], ReadSchema: struct<uid:int,dt:date,suid:int>
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint))), [id=#233]
+- *(1) Project [pid#11, vs#12, uid#13]
+- *(1) Filter isnotnull(uid#13)
+- *(1) ColumnarToRow
+- FileScan parquet default.t2[pid#11,vs#12,uid#13] Batched: true, DataFilters: [isnotnull(uid#13)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-bin-hadoop3.2/spark-warehouse/t2], PartitionFilters: [], PushedFilters: [IsNotNull(uid)], ReadSchema: struct<pid:int,vs:int,uid:int>
```
Plan. after. this pr. :
```
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- HashAggregate(keys=[years#0, appversion#2], functions=[sum(uusers#3L)], output=[years#0, appversion#2, users#5L])
+- Exchange hashpartitioning(years#0, appversion#2, 5), true, [id=#71]
+- HashAggregate(keys=[years#0, appversion#2], functions=[partial_sum(uusers#3L)], output=[years#0, appversion#2, sum#22L])
+- HashAggregate(keys=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12], functions=[count(distinct uid#7)], output=[years#0, appversion#2, uusers#3L])
+- Exchange hashpartitioning(date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, 5), true, [id=#67]
+- HashAggregate(keys=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12], functions=[partial_count(distinct uid#7)], output=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, count#27L])
+- HashAggregate(keys=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7], functions=[], output=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7])
+- Exchange hashpartitioning(date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7, 5), true, [id=#63]
+- HashAggregate(keys=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles)) AS date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END AS CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7], functions=[], output=[date_trunc(year, cast(dt#9 as timestamp), Some(America/Los_Angeles))#23, CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END#24, vs#12, uid#7])
+- Project [uid#7, dt#9, pid#11, vs#12]
+- BroadcastHashJoin [uid#7], [uid#13], Inner, BuildRight, false
:- Filter isnotnull(uid#7)
: +- FileScan parquet default.t1[uid#7,dt#9] Batched: true, DataFilters: [isnotnull(uid#7)], Format: Parquet, Location: InMemoryFileIndex[file:/private/var/folders/4l/7_c5c97s1_gb0d9_d6shygx00000gn/T/warehouse-c069d87..., PartitionFilters: [], PushedFilters: [IsNotNull(uid)], ReadSchema: struct<uid:int,dt:date>
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[2, int, false] as bigint)),false), [id=#58]
+- Filter ((CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END = iOS) AND isnotnull(uid#13))
+- FileScan parquet default.t2[pid#11,vs#12,uid#13] Batched: true, DataFilters: [(CASE WHEN (pid#11 = 3) THEN iOS WHEN (pid#11 = 4) THEN Android ELSE Other END = iOS), isnotnull..., Format: Parquet, Location: InMemoryFileIndex[file:/private/var/folders/4l/7_c5c97s1_gb0d9_d6shygx00000gn/T/warehouse-c069d87..., PartitionFilters: [], PushedFilters: [IsNotNull(uid)], ReadSchema: struct<pid:int,vs:int,uid:int>
```
### Why are the changes needed?
Improve performance, filter more data.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added UT
Closes#30278 from AngersZhuuuu/SPARK-33302.
Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This changes `DropTableExec` to also invalidate caches referencing the table to be dropped, in a cascading manner.
### Why are the changes needed?
In DSv1, `DROP TABLE` command also invalidate caches as described in [SPARK-19765](https://issues.apache.org/jira/browse/SPARK-19765). However in DSv2 the same command only drops the table but doesn't handle the caches. This could lead to correctness issue.
### Does this PR introduce _any_ user-facing change?
Yes. Now DSv2 `DROP TABLE` command also invalidates cache.
### How was this patch tested?
Added a new UT
Closes#30211 from sunchao/SPARK-33305.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
When a system.exit exception occurs during the process, the python worker exits abnormally, and then the executor task is still waiting for the worker for reading from socket, causing it to hang.
The system.exit exception may be caused by the user's error code, but spark should at least throw an error to remind the user, not get stuck
we can run a simple test to reproduce this case:
```
from pyspark.sql import SparkSession
def err(line):
raise SystemExit
spark = SparkSession.builder.appName("test").getOrCreate()
spark.sparkContext.parallelize(range(1,2), 2).map(err).collect()
spark.stop()
```
### Why are the changes needed?
to make sure pyspark application won't hang if there's non-Exception error in python worker
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
added a new test and also manually tested the case above
Closes#30248 from li36909/pyspark.
Lead-authored-by: lrz <lrz@lrzdeMacBook-Pro.local>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
element_at with CreateArray not respect one based index.
repo step:
```
var df = spark.sql("select element_at(array(3, 2, 1), 0)")
df.printSchema()
df = spark.sql("select element_at(array(3, 2, 1), 1)")
df.printSchema()
df = spark.sql("select element_at(array(3, 2, 1), 2)")
df.printSchema()
df = spark.sql("select element_at(array(3, 2, 1), 3)")
df.printSchema()
root
– element_at(array(3, 2, 1), 0): integer (nullable = false)
root
– element_at(array(3, 2, 1), 1): integer (nullable = false)
root
– element_at(array(3, 2, 1), 2): integer (nullable = false)
root
– element_at(array(3, 2, 1), 3): integer (nullable = true)
correct answer should be
0 true which is outOfBounds return default true.
1 false
2 false
3 false
```
For expression eval, it respect the oneBasedIndex, but within checking the nullable, it calculates with zeroBasedIndex using `computeNullabilityFromArray`.
### Why are the changes needed?
Correctness issue.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Added UT and existing UT.
Closes#30296 from leanken/leanken-SPARK-33391.
Authored-by: xuewei.linxuewei <xuewei.linxuewei@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
- Call `spark.read.table` in `spark.table`.
- Add comments for `spark.table` to emphasize it also support streaming temp view reading.
### Why are the changes needed?
The code paths of `spark.table` and `spark.read.table` should be the same. This behavior is broke in SPARK-32592 since we need to respect options in `spark.read.table` API.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UT.
Closes#30148 from xuanyuanking/SPARK-33244.
Authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR proposes to migrate `LOAD DATA` to use `UnresolvedTable` to resolve the table identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).
Note that `LOAD DATA` is not supported for v2 tables.
### Why are the changes needed?
The changes allow consistent resolution behavior when resolving the table identifier. For example, the following is the current behavior:
```scala
sql("CREATE TEMPORARY VIEW t AS SELECT 1")
sql("CREATE DATABASE db")
sql("CREATE TABLE t (key INT, value STRING) USING hive")
sql("USE db")
sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE t") // Succeeds
```
With this change, `LOAD DATA` above fails with the following:
```
org.apache.spark.sql.AnalysisException: t is a temp view not table.; line 1 pos 0
at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews$$anonfun$apply$7.$anonfun$applyOrElse$39(Analyzer.scala:865)
at scala.Option.foreach(Option.scala:407)
```
, which is expected since temporary view is resolved first and `LOAD DATA` doesn't support a temporary view.
### Does this PR introduce _any_ user-facing change?
After this PR, `LOAD DATA ... t` is resolved to a temp view `t` instead of table `db.t` in the above scenario.
### How was this patch tested?
Updated existing tests.
Closes#30270 from imback82/load_data_cmd.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
When TableProvider.supportsExternalMetadata() is true, Spark will use the input Dataframe's schema in `DataframeWriter.save()`/`DataStreamWriter.start()` and skip schema/partitioning inference.
### Why are the changes needed?
For all the v2 data sources which are not FileDataSourceV2, Spark always infers the table schema/partitioning on `DataframeWriter.save()`/`DataStreamWriter.start()`.
The inference of table schema/partitioning can be expensive. However, there is no such trait or flag for indicating a V2 source can use the input DataFrame's schema on `DataframeWriter.save()`/`DataStreamWriter.start()`. We can resolve the problem by adding a new expected behavior for the method `TableProvider.supportsExternalMetadata()`.
### Does this PR introduce _any_ user-facing change?
Yes, a new behavior for the data source v2 API `TableProvider.supportsExternalMetadata()` when it returns true.
### How was this patch tested?
Unit test
Closes#30273 from gengliangwang/supportsExternalMetadata.
Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This upgrade Apache Arrow version from 1.0.1 to 2.0.0
### Why are the changes needed?
Apache Arrow 2.0.0 was released with some improvements from Java side, so it's better to upgrade Spark to the new version.
Note that the format version in Arrow 2.0.0 is still 1.0.0 so API should still be compatible between 1.0.1 and 2.0.0.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Existing UTs.
Closes#30306 from sunchao/SPARK-33213.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Structured Streaming UI is not containing state information. In this PR I've added it.
### Why are the changes needed?
Missing state information.
### Does this PR introduce _any_ user-facing change?
Additional UI elements appear.
### How was this patch tested?
Existing unit tests + manual test.
<img width="1044" alt="Screenshot 2020-10-30 at 15 14 21" src="https://user-images.githubusercontent.com/18561820/97715405-a1797000-1ac2-11eb-886a-e3e6efa3af3e.png">
Closes#30151 from gaborgsomogyi/SPARK-33223.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
add prompt information about current applicationId, current URL and master info when pyspark / sparkR starts.
### Why are the changes needed?
The information printed when pyspark/sparkR starts does not prompt the basic information of current application, and it is not convenient when used pyspark/sparkR in dos.
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
manual test result shows below:
![pyspark new print](https://user-images.githubusercontent.com/52202080/98274268-2a663f00-1fce-11eb-88ce-964ce90b439e.png)
![sparkR](https://user-images.githubusercontent.com/52202080/98541235-1a01dd00-22ca-11eb-9304-09bcde87b05e.png)
Closes#30266 from akiyamaneko/pyspark-hint-info.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR aims to upgrade `commons-compress` from 1.8 to 1.20.
### Why are the changes needed?
- https://commons.apache.org/proper/commons-compress/security-reports.html
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CIs.
Closes#30304 from dongjoon-hyun/SPARK-33405.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR proposes migration of `pyspark.ml` to NumPy documentation style.
### Why are the changes needed?
To improve documentation style.
### Does this PR introduce _any_ user-facing change?
Yes, this changes both rendered HTML docs and console representation (SPARK-33243).
### How was this patch tested?
`dev/lint-python` and manual inspection.
Closes#30285 from zero323/SPARK-33251.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
Update the package commons-crypto to v1.1.0 to support aarch64 platform
- https://issues.apache.org/jira/browse/CRYPTO-139
### Why are the changes needed?
The package commons-crypto-1.0.0 available in the Maven repository
doesn't support aarch64 platform. It costs long time in
CryptoRandomFactory.getCryptoRandom(properties).nextBytes(iv) when NettyBlockRpcSever
receive block data from client, if the time more than the default value 120s, IOException raised and client
will retry replicate the block data to other executors. But in fact the replication is complete,
it makes the replication number incorrect.
This makes DistributedSuite tests pass.
### Does this PR introduce any user-facing change?
No
### How was this patch tested?
Pass the CIs.
Closes#30275 from huangtianhua/SPARK-32691.
Authored-by: huangtianhua <huangtianhua223@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
### What changes were proposed in this pull request?
This is one of the patches for SPIP [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602) which is needed for push-based shuffle.
Summary of changes:
- Adds an implementation of `MergedShuffleFileManager` which was introduced with [Spark 32915](https://issues.apache.org/jira/browse/SPARK-32915).
- Integrated the push-based shuffle service with `YarnShuffleService`.
### Why are the changes needed?
Refer to the SPIP in [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Added unit tests.
The reference PR with the consolidated changes covering the complete implementation is also provided in [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).
We have already verified the functionality and the improved performance as documented in the SPIP doc.
Lead-authored-by: Min Shen mshenlinkedin.com
Co-authored-by: Chandni Singh chsinghlinkedin.com
Co-authored-by: Ye Zhou yezhoulinkedin.com
Closes#30062 from otterc/SPARK-32916.
Lead-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Co-authored-by: Ye Zhou <yezhou@linkedin.com>
Co-authored-by: Min Shen <mshen@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
This PR modifies the `ExtractPythonUDFs` rule to deduplicate deterministic PythonUDF calls.
Before this PR the dataframe: `df.withColumn("c", batchedPythonUDF(col("a"))).withColumn("d", col("c"))` has the plan:
```
*(1) Project [value#1 AS a#4, pythonUDF1#15 AS c#7, pythonUDF1#15 AS d#10]
+- BatchEvalPython [dummyUDF(value#1), dummyUDF(value#1)], [pythonUDF0#14, pythonUDF1#15]
+- LocalTableScan [value#1]
```
After this PR the deterministic PythonUDF calls are deduplicated:
```
*(1) Project [value#1 AS a#4, pythonUDF0#14 AS c#7, pythonUDF0#14 AS d#10]
+- BatchEvalPython [dummyUDF(value#1)], [pythonUDF0#14]
+- LocalTableScan [value#1]
```
### Why are the changes needed?
To fix a performance issue.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
New and existing UTs.
Closes#30203 from peter-toth/SPARK-33303-deduplicate-deterministic-udf-calls.
Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
After #30097, all rules are using `SparkSession.active` to get `SQLConf`
and `SparkSession`. But in AQE, when applying the rules for the initial plan,
we should use the spark session in AQE context.
### Why are the changes needed?
Fix potential problem caused by using the wrong spark session
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Existing ut
Closes#30294 from linhongliu-db/SPARK-33140-followup.
Authored-by: Linhong Liu <linhong.liu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This pr add support bucket pruning on `IsNaN` predicate.
### Why are the changes needed?
Improve query performance.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Unit test.
Closes#30291 from wangyum/SPARK-33385.
Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR makes `DataFrameWriterV2` to create query plans with `UnresolvedRelation` and leave the table resolution work to the analyzer.
### Why are the changes needed?
Table resolution work should be done by the analyzer. After this PR, the behavior is more consistent between different APIs (DataFrameWriter, DataFrameWriterV2 and SQL). See the next section for behavior changes.
### Does this PR introduce _any_ user-facing change?
Yes.
1. writes to a temp view of v2 relation: previously it fails with table not found exception, now it works if the v2 relation is writable. This is consistent with `DataFrameWriter` and SQL INSERT.
2. writes to other temp views: previously it fails with table not found exception, now it fails with a more explicit error message, saying that writing to a temp view of non-v2-relation is not allowed.
3. writes to a view: previously it fails with table not writable error, now it fails with a more explicit error message, saying that writing to a view is not allowed.
4. writes to a v1 table: previously it fails with table not writable error, now it fails with a more explicit error message, saying that writing to a v1 table is not allowed. (We can allow it later, by falling back to v1 command)
### How was this patch tested?
new tests
Closes#29970 from cloud-fan/refactor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
Currently in JDBCTableCatalog, we ignore the table options when creating table.
```
// TODO (SPARK-32405): Apply table options while creating tables in JDBC Table Catalog
if (!properties.isEmpty) {
logWarning("Cannot create JDBC table with properties, these properties will be " +
"ignored: " + properties.asScala.map { case (k, v) => s"$k=$v" }.mkString("[", ", ", "]"))
}
```
### Why are the changes needed?
need to apply the table options when we create table
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
add new test
Closes#30154 from huaxingao/table_options.
Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR aims to support sorted shuffle block migration.
### Why are the changes needed?
Since the current shuffle block migration works in a random order, the failure during worker decommission affects all shuffles. We had better finish the shuffles one by one to minimize the number of affected shuffle.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CIs with the newly added test case.
Closes#30293 from dongjoon-hyun/SPARK-33387.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
In `RenameBasedFSDataOutputStream.cancel`, we do two things: closing underlying stream and delete temporary file, in a single try/catch block. Closing `OutputStream` could possibly throw `IOException` so we possibly missing deleting temporary file.
This patch proposes to delete temporary even underlying stream throwing error.
### Why are the changes needed?
To avoid leaving temporary files during canceling writing in `RenameBasedFSDataOutputStream`.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test.
Closes#30290 from viirya/SPARK-33384.
Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
There are two similar compilation warnings about procedure-like declaration in Scala 2.13:
```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:70: procedure syntax is deprecated for constructors: add `=`, as in method definition
```
and
```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala:211: procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `run`'s return type
```
this pr is the first part to resolve SPARK-33352:
- For constructors method definition add `=` to convert to function syntax
- For without `return type` methods definition add `: Unit =` to convert to function syntax
### Why are the changes needed?
Eliminate compilation warnings in Scala 2.13 and this change should be compatible with Scala 2.12
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30255 from LuciferYang/SPARK-29392-FOLLOWUP.1.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
### What changes were proposed in this pull request?
Javadocs updated for the encoder to include maps as a collection type
### Why are the changes needed?
The javadocs were not updated with fix SPARK-16706
### Does this PR introduce _any_ user-facing change?
Yes, the javadocs are updated
### How was this patch tested?
sbt was run to ensure it meets scalastyle
Closes#30274 from hannahkamundson/SPARK-32860.
Lead-authored-by: Hannah Amundson <amundson.hannah@heb.com>
Co-authored-by: Hannah <48397717+hannahkamundson@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
There are 4 fields in `MutableApplicationInfo ` seems useless:
- `coresGranted`
- `maxCores`
- `coresPerExecutor`
- `memoryPerExecutorMB`
They are always `None` and not reassigned.
So the main change of this pr is cleanup these useless fields in `MutableApplicationInfo`.
### Why are the changes needed?
Cleanup useless variables.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Pass the Jenkins or GitHub Action
Closes#30251 from LuciferYang/SPARK-33347.
Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
The changes in [SPARK-32501 Inconsistent NULL conversions to strings](https://issues.apache.org/jira/browse/SPARK-32501) introduced some behavior that I'd like to clean up a bit.
Here's sample code to illustrate the behavior I'd like to clean up:
```scala
val rows = Seq[String](null)
.toDF("value")
.withColumn("struct1", struct('value as "value1"))
.withColumn("struct2", struct('value as "value1", 'value as "value2"))
.withColumn("array1", array('value))
.withColumn("array2", array('value, 'value))
// Show the DataFrame using the "first" codepath.
rows.show(truncate=false)
+-----+-------+-------------+------+--------+
|value|struct1|struct2 |array1|array2 |
+-----+-------+-------------+------+--------+
|null |{ null}|{ null, null}|[] |[, null]|
+-----+-------+-------------+------+--------+
// Write the DataFrame to disk, then read it back and show it to trigger the "codegen" code path:
rows.write.parquet("rows")
spark.read.parquet("rows").show(truncate=false)
+-----+-------+-------------+-------+-------------+
|value|struct1|struct2 |array1 |array2 |
+-----+-------+-------------+-------+-------------+
|null |{ null}|{ null, null}|[ null]|[ null, null]|
+-----+-------+-------------+-------+-------------+
```
Notice:
1. If the first element of a struct is null, it is printed with a leading space (e.g. "\{ null\}"). I think it's preferable to print it without the leading space (e.g. "\{null\}"). This is consistent with how non-null values are printed inside a struct.
2. If the first element of an array is null, it is not printed at all in the first code path, and the "codegen" code path prints it with a leading space. I think both code paths should be consistent and print it without a leading space (e.g. "[null]").
The desired result of this PR is to product the following output via both code paths:
```
+-----+-------+------------+------+------------+
|value|struct1|struct2 |array1|array2 |
+-----+-------+------------+------+------------+
|null |{null} |{null, null}|[null]|[null, null]|
+-----+-------+------------+------+------------+
```
This contribution is my original work and I license the work to the project under the project’s open source license.
### Why are the changes needed?
To correct errors and inconsistencies in how DataFrame.show() displays nulls inside arrays and structs.
### Does this PR introduce _any_ user-facing change?
Yes. This PR changes what is printed out by DataFrame.show().
### How was this patch tested?
I added new test cases in CastSuite.scala to cover the cases addressed by this PR.
Closes#30189 from stwhit/show_nulls.
Authored-by: Stuart White <stuart.white1@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
### What changes were proposed in this pull request?
Add executor peak jvm memory metrics in executors page
![image](https://user-images.githubusercontent.com/1633312/97767765-9121bf00-1adb-11eb-93c7-7912d9fe7826.png)
### Why are the changes needed?
Users can know executor peak jvm metrics on in executors page
### Does this PR introduce _any_ user-facing change?
Users can know executor peak jvm metrics on in executors page
### How was this patch tested?
Manually tested
Closes#30186 from warrenzhu25/23432.
Authored-by: Warren Zhu <warren.zhu25@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
### What changes were proposed in this pull request?
This PR proposes to introduce the `purge` option in `TableCatalog.dropTable` so that v2 catalogs can use the option if needed.
Related discussion: https://github.com/apache/spark/pull/30079#discussion_r510594110
### Why are the changes needed?
Spark DDL supports passing the purge option to `DROP TABLE` command. However, the option is not used (ignored) for v2 catalogs.
### Does this PR introduce _any_ user-facing change?
This PR introduces a new API in `TableCatalog`.
### How was this patch tested?
Added a test.
Closes#30267 from imback82/purge_table.
Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Override the default SQL strings for:
ALTER TABLE RENAME COLUMN
ALTER TABLE UPDATE COLUMN NULLABILITY
in the following MsSQLServer JDBC dialect according to official documentation.
Write MsSqlServer integration tests for JDBC.
### Why are the changes needed?
To add the support for alter table when interacting with MSSql Server.
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
added tests
Closes#30038 from ScrapCodes/mssql-dialect.
Authored-by: Prashant Sharma <prashsh1@in.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
fix the wrong url and display name of blocking thread in threadDump page.
The blockingThreadId variable passed to the page should be of string type instead of Option type.
### Why are the changes needed?
blocking threadId in the ui page is not displayed well, and the corresponding url cannot be redirected normally
### Does this PR introduce _any_ user-facing change?
NO
### How was this patch tested?
The pr only involves minor changes to the page and does not affect other functions,
The manual test results are as follows. The thread name displayed on the page is correct, and you can click on the URL to jump to the corresponding url
![shows_ok](https://user-images.githubusercontent.com/52202080/98108177-89488d00-1ed6-11eb-9488-8446c3f38bad.gif)
Closes#30249 from akiyamaneko/thread-dump-improve.
Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
### What changes were proposed in this pull request?
1. Rename `OffsetWindowSpec` to `OffsetWindowFunction`, as it's the base class for all offset based window functions.
2. Refine and add more comments.
3. Remove `isRelative` as it's useless.
### Why are the changes needed?
code refinement
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
existing tests
Closes#30261 from cloud-fan/window.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
This PR fixes incorrect JIRA ids in `Analyzer.scala` introduced by SPARK-31670 (https://github.com/apache/spark/pull/28490)
```scala
- // SPARK-31607: Resolve Struct field in selectedGroupByExprs/groupByExprs and aggregations
+ // SPARK-31670: Resolve Struct field in selectedGroupByExprs/groupByExprs and aggregations
```
### Why are the changes needed?
Fix the wrong information.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
This is a comment change. Manually review.
Closes#30269 from dongjoon-hyun/SPARK-31670-MINOR.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
### What changes were proposed in this pull request?
This PR aims to update SBT from 1.4.1 to 1.4.2.
### Why are the changes needed?
This will bring the latest bug fixes.
- https://github.com/sbt/sbt/releases/tag/v1.4.2
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Pass the CIs.
Closes#30268 from williamhyun/sbt.
Authored-by: William Hyun <williamhyun3@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Removing duplicated code in `ResolveOutputRelation`, by adding `V2WriteCommand.withNewQuery`
### Why are the changes needed?
code cleanup
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
existing tests
Closes#30264 from cloud-fan/ds-minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Currently when run in `cluster` mode on YARN, the Spark `yarn.Client` will print out the application report into the logs, to be easily viewed by users. For example:
```
INFO yarn.Client:
client token: Token { kind: YARN_CLIENT_TOKEN, service: }
diagnostics: N/A
ApplicationMaster host: X.X.X.X
ApplicationMaster RPC port: 0
queue: default
start time: 1602782566027
final status: UNDEFINED
tracking URL: http://hostname:8888/proxy/application_<id>/
user: xkrogen
```
I propose adding, alongside the application report, some additional lines like:
```
Driver Logs (stdout): http://hostname:8042/node/containerlogs/container_<id>/xkrogen/stdout?start=-4096
Driver Logs (stderr): http://hostname:8042/node/containerlogs/container_<id>/xkrogen/stderr?start=-4096
```
This information isn't contained in the `ApplicationReport`, so it's necessary to query the ResourceManager REST API. For now I have added this as an always-on feature, but if there is any concern about adding this REST dependency, I think hiding this feature behind an off-by-default flag is reasonable.
### Why are the changes needed?
Typically, the tracking URL can be used to find the logs of the ApplicationMaster/driver while the application is running. Later, the Spark History Server can be used to track this information down, using the stdout/stderr links on the Executors page.
However, in the situation when the driver crashed _before_ writing out a history file, the SHS may not be aware of this application, and thus does not contain links to the driver logs. When this situation arises, it can be difficult for users to debug further, since they can't easily find their driver logs.
It is possible to reach the logs by using the `yarn logs` commands, but the average Spark user isn't aware of this and shouldn't have to be.
With this information readily available in the logs, users can quickly jump to their driver logs, even if it crashed before the SHS became aware of the application. This has the additional benefit of providing a quick way to access driver logs, which often contain useful information, in a single click (instead of navigating through the Spark UI).
### Does this PR introduce _any_ user-facing change?
Yes, some additional print statements will be created in the application report when using YARN in cluster mode.
### How was this patch tested?
Added unit tests for the parsing logic in `yarn.ClientSuite`. Also tested against a live cluster. When the driver is running:
```
INFO Client: Application report for application_XXXXXXXXX_YYYYYY (state: RUNNING)
INFO Client:
client token: Token { kind: YARN_CLIENT_TOKEN, service: }
diagnostics: N/A
ApplicationMaster host: host.example.com
ApplicationMaster RPC port: ######
queue: queue_name
start time: 1604529046091
final status: UNDEFINED
tracking URL: http://host.example.com:8080/proxy/application_XXXXXXXXX_YYYYYY/
user: xkrogen
Driver Logs (stdout): http://host.example.com:8042/node/containerlogs/container_e07_XXXXXXXXX_YYYYYY_01_000001/xkrogen/stdout?start=-4096
Driver Logs (stderr): http://host.example.com:8042/node/containerlogs/container_e07_XXXXXXXXX_YYYYYY_01_000001/xkrogen/stderr?start=-4096
INFO Client: Application report for application_XXXXXXXXX_YYYYYY (state: RUNNING)
```
I confirmed that when the driver has not yet launched, the report does not include the two Driver Logs items. Will omit the output here for brevity since it looks the same.
Closes#30096 from xkrogen/xkrogen-SPARK-33185-yarn-client-print.
Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
### What changes were proposed in this pull request?
Update SQL migration guide for SPARK-33290
### Why are the changes needed?
Make the change better documented.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
N/A
Closes#30256 from sunchao/SPARK-33290-2.
Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
This PR change the behavior of GitHub Actions job that caches dependencies.
SPARK-33226 upgraded sbt to 1.4.1.
As of 1.3.0, sbt uses Coursier as the dependency resolver / fetcher.
So let's change the dependency cache configuration for the GitHub Actions job.
### Why are the changes needed?
To make build faster with Coursier for the GitHub Actions job.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Should be done by GitHub Actions itself.
Closes#30259 from sarutak/coursier-cache.
Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
Fix a small bug in `V2WriteCommand.resolved`. It should always require the `table` and `query` to be resolved.
### Why are the changes needed?
To prevent potential bugs that we skip resolve the input query.
### Does this PR introduce _any_ user-facing change?
no
### How was this patch tested?
a new test
Closes#30265 from cloud-fan/ds-minor-2.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
### What changes were proposed in this pull request?
There's a concept of 'read-only' and 'read+write' state store in Spark which is defined "implicitly". Spark doesn't prevent write for 'read-only' state store; Spark just assumes read-only stateful operator will not modify the state store. Given it's not defined explicitly, the instance of state store has to be implemented as 'read+write' even it's being used as 'read-only', which sometimes brings confusion.
For example, abort() in HDFSBackedStateStore - d38f816748/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala (L143-L155)
The comment sounds as if statement works differently between 'read-only' and 'read+write', but that's not true as both state store has state initialized as UPDATING (no difference). So 'read-only' state also creates the temporary file, initializes output streams to write to temporary file, closes output streams, and finally deletes the temporary file. This unnecessary operations are being done per batch/partition.
This patch explicitly defines 'read-only' StateStore, and enables state store provider to create 'read-only' StateStore instance if requested. Relevant code paths are modified, as well as 'read-only' StateStore implementation for HDFSBackedStateStore is introduced. The new implementation gets rid of unnecessary operations explained above.
In point of backward-compatibility view, the only thing being changed in public API side is `StateStoreProvider`. The trait `StateStoreProvider` has to be changed to allow requesting 'read-only' StateStore; this patch adds default implementation which leverages 'read+write' StateStore but wrapping with 'write-protected' StateStore instance, so that custom providers don't need to change their code to reflect the change. But if the providers can optimize for read-only workload, they'll be happy to make a change.
Please note that this patch makes ReadOnlyStateStore extend StateStore and being referred as StateStore, as StateStore is being used in so many places and it's not easy to support both traits if we differentiate them. So unfortunately these write methods are still exposed for read-only state; it just throws UnsupportedOperationException.
### Why are the changes needed?
The new API opens the chance to optimize read-only state store instance compared with read+write state store instance. HDFSBackedStateStoreProvider is modified to provide read-only version of state store which doesn't deal with temporary file as well as state machine.
### Does this PR introduce any user-facing change?
Clearly "no" for most end users, and also "no" for custom state store providers as it doesn't touch trait `StateStore` as well as provides default implementation for added method in trait `StateStoreProvider`.
### How was this patch tested?
Modified UT. Existing UTs ensure the change doesn't break anything.
Closes#26935 from HeartSaVioR/SPARK-30294.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>