[SPARK-30532] DataFrameStatFunctions to work with TABLE.COLUMN syntax#27916
[SPARK-30532] DataFrameStatFunctions to work with TABLE.COLUMN syntax#27916kachayev wants to merge 8 commits intoapache:masterfrom
Conversation
|
ok to test |
|
Test build #119807 has finished for PR 27916 at commit
|
…s not throw exception
|
Fixed tests. |
|
Test build #119830 has finished for PR 27916 at commit
|
|
Tests failed because of something that seems completely irrelevant (in |
|
retest this please |
There was a problem hiding this comment.
can we merge these tests into one? Then we can share the below code
val df1 = spark.sparkContext.parallelize(0 to 10).toDF("num").as("table1")
val df2 = spark.sparkContext.parallelize(0 to 10).toDF("num").as("table2")
val dfx = df2.crossJoin(df1)
There was a problem hiding this comment.
Sure, I will submit update shortly.
|
Test build #120044 has finished for PR 27916 at commit
|
|
Test build #120149 has finished for PR 27916 at commit
|
|
Looks like tests failed for the same reason as previously. Could not replicate this locally (local build & tests are just fine). |
|
retest this please |
|
Test build #120176 has finished for PR 27916 at commit
|
|
retest this please |
|
Test build #120185 has finished for PR 27916 at commit
|
|
retest this please |
|
Test build #120207 has finished for PR 27916 at commit
|
|
Test build #120561 has finished for PR 27916 at commit
|
|
@cloud-fan Merged with latest master. Now tests work (previously they failed with unrelated issues). |
|
thanks, merging to master/3.0! |
### What changes were proposed in this pull request?
`DataFrameStatFunctions` now works correctly with fully qualified column name (Table.Column syntax) by properly resolving the name instead of relying on field names from schema, notably:
* `approxQuantile`
* `freqItems`
* `cov`
* `corr`
(other functions from `DataFrameStatFunctions` already work correctly).
See code examples below.
### Why are the changes needed?
With current implementation some stat functions are impossible to use when joining datasets with similar column names.
### Does this PR introduce any user-facing change?
Yes. Before the change, the following code would fail with `AnalysisException`.
```scala
scala> val df1 = sc.parallelize(0 to 10).toDF("num").as("table1")
df1: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [num: int]
scala> val df2 = sc.parallelize(0 to 10).toDF("num").as("table2")
df2: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [num: int]
scala> val dfx = df2.crossJoin(df1)
dfx: org.apache.spark.sql.DataFrame = [num: int, num: int]
scala> dfx.stat.approxQuantile("table1.num", Array(0.1), 0.0)
res0: Array[Double] = Array(1.0)
scala> dfx.stat.corr("table1.num", "table2.num")
res1: Double = 1.0
scala> dfx.stat.cov("table1.num", "table2.num")
res2: Double = 11.0
scala> dfx.stat.freqItems(Array("table1.num", "table2.num"))
res3: org.apache.spark.sql.DataFrame = [table1.num_freqItems: array<int>, table2.num_freqItems: array<int>]
```
### How was this patch tested?
Corresponding unit tests are added to `DataFrameStatSuite.scala` (marked as "SPARK-30532").
Closes #27916 from kachayev/fix-spark-30532.
Authored-by: Oleksii Kachaiev <kachayev@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 22bb6b0)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
### What changes were proposed in this pull request?
`DataFrameStatFunctions` now works correctly with fully qualified column name (Table.Column syntax) by properly resolving the name instead of relying on field names from schema, notably:
* `approxQuantile`
* `freqItems`
* `cov`
* `corr`
(other functions from `DataFrameStatFunctions` already work correctly).
See code examples below.
### Why are the changes needed?
With current implementation some stat functions are impossible to use when joining datasets with similar column names.
### Does this PR introduce any user-facing change?
Yes. Before the change, the following code would fail with `AnalysisException`.
```scala
scala> val df1 = sc.parallelize(0 to 10).toDF("num").as("table1")
df1: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [num: int]
scala> val df2 = sc.parallelize(0 to 10).toDF("num").as("table2")
df2: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [num: int]
scala> val dfx = df2.crossJoin(df1)
dfx: org.apache.spark.sql.DataFrame = [num: int, num: int]
scala> dfx.stat.approxQuantile("table1.num", Array(0.1), 0.0)
res0: Array[Double] = Array(1.0)
scala> dfx.stat.corr("table1.num", "table2.num")
res1: Double = 1.0
scala> dfx.stat.cov("table1.num", "table2.num")
res2: Double = 11.0
scala> dfx.stat.freqItems(Array("table1.num", "table2.num"))
res3: org.apache.spark.sql.DataFrame = [table1.num_freqItems: array<int>, table2.num_freqItems: array<int>]
```
### How was this patch tested?
Corresponding unit tests are added to `DataFrameStatSuite.scala` (marked as "SPARK-30532").
Closes apache#27916 from kachayev/fix-spark-30532.
Authored-by: Oleksii Kachaiev <kachayev@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
What changes were proposed in this pull request?
DataFrameStatFunctionsnow works correctly with fully qualified column name (Table.Column syntax) by properly resolving the name instead of relying on field names from schema, notably:approxQuantilefreqItemscovcorr(other functions from
DataFrameStatFunctionsalready work correctly).See code examples below.
Why are the changes needed?
With current implementation some stat functions are impossible to use when joining datasets with similar column names.
Does this PR introduce any user-facing change?
Yes. Before the change, the following code would fail with
AnalysisException.How was this patch tested?
Corresponding unit tests are added to
DataFrameStatSuite.scala(marked as "SPARK-30532").