Skip to content
Closed
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions R/pkg/NAMESPACE
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,7 @@ exportMethods("arrange",
"toJSON",
"transform",
"union",
"unionAll",
"unionByName",
"unique",
"unpersist",
Expand Down
14 changes: 14 additions & 0 deletions R/pkg/R/DataFrame.R
Original file line number Diff line number Diff line change
Expand Up @@ -2732,6 +2732,20 @@ setMethod("union",
dataFrame(unioned)
})

#' Return a new SparkDataFrame containing the union of rows
#'
#' This is an alias for `union`.
Copy link
Member

@felixcheung felixcheung Nov 27, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the goal is for this to be like other *All, this should go into a separate doc page, plus seealso, example etc.

The way this was written, as it was a deprecated function, this doc page merged with union - as it is committed now, none of the text above will show up and also unionAll will not be listed in method index list.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also backtick doesn't format with roxygen2. this should be

This is an alias for \code{union}.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see. Instead of directly copying the comments back, we should follow intersectAll. Opened a ticket: https://issues.apache.org/jira/browse/SPARK-26189

#'
#' @rdname union
#' @name unionAll
#' @aliases unionAll,SparkDataFrame,SparkDataFrame-method
#' @note unionAll since 1.4.0
setMethod("unionAll",
signature(x = "SparkDataFrame", y = "SparkDataFrame"),
function(x, y) {
union(x, y)
})

#' Return a new SparkDataFrame containing the union of rows, matched by column names
#'
#' Return a new SparkDataFrame containing the union of rows in this SparkDataFrame
Expand Down
3 changes: 3 additions & 0 deletions R/pkg/R/generics.R
Original file line number Diff line number Diff line change
Expand Up @@ -631,6 +631,9 @@ setGeneric("toRDD", function(x) { standardGeneric("toRDD") })
#' @rdname union
setGeneric("union", function(x, y) { standardGeneric("union") })

#' @rdname union
setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") })

#' @rdname unionByName
setGeneric("unionByName", function(x, y) { standardGeneric("unionByName") })

Expand Down
1 change: 1 addition & 0 deletions R/pkg/tests/fulltests/test_sparkSQL.R
Original file line number Diff line number Diff line change
Expand Up @@ -2458,6 +2458,7 @@ test_that("union(), unionByName(), rbind(), except(), and intersect() on a DataF
expect_equal(count(unioned), 6)
expect_equal(first(unioned)$name, "Michael")
expect_equal(count(arrange(suppressWarnings(union(df, df2)), df$age)), 6)
expect_equal(count(arrange(suppressWarnings(unionAll(df, df2)), df$age)), 6)

df1 <- select(df2, "age", "name")
unioned1 <- arrange(unionByName(df1, df), df1$age)
Expand Down
2 changes: 1 addition & 1 deletion docs/sparkr.md
Original file line number Diff line number Diff line change
Expand Up @@ -718,4 +718,4 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma
## Upgrading to SparkR 3.0.0

- The deprecated methods `sparkR.init`, `sparkRSQL.init`, `sparkRHive.init` have been removed. Use `sparkR.session` instead.
- The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `registerTempTable`, `createExternalTable`, `dropTempTable`, `unionAll` have been removed. Use `read.parquet`, `write.parquet`, `read.json`, `createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead.
- The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `registerTempTable`, `createExternalTable`, and `dropTempTable` have been removed. Use `read.parquet`, `write.parquet`, `read.json`, `createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead.
2 changes: 0 additions & 2 deletions docs/sql-migration-guide-upgrade.md
Original file line number Diff line number Diff line change
Expand Up @@ -341,8 +341,6 @@ displayTitle: Spark SQL Upgrading Guide
APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the
single-node data frame notion in these languages.

- Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union`
Copy link
Member

@dongjoon-hyun dongjoon-hyun Nov 25, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ur, we cannot change the history. Until Spark 2.4.0, we are showing the deprecation warning.

scala> spark.version
res2: String = 2.4.0

scala> df.unionAll(df2)
<console>:28: warning: method unionAll in class Dataset is deprecated: use union()
       df.unionAll(df2)
          ^

Shall we keep the history in this specific migration doc, Upgrading From Spark SQL 1.6 to 2.0, and add some comment about 3.0.0 instead?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's my fault for making this suggestion. Yeah maybe best to leave this statement, and add a note here or the the 3.0 migration guide that it has been subsequently un-deprecated


- Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap`

- Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView`
Expand Down
11 changes: 11 additions & 0 deletions python/pyspark/sql/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -1448,6 +1448,17 @@ def union(self, other):
"""
return DataFrame(self._jdf.union(other._jdf), self.sql_ctx)

@since(1.3)
def unionAll(self, other):
""" Return a new :class:`DataFrame` containing union of rows in this and another frame.

This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union
(that does deduplication of elements), use this function followed by :func:`distinct`.

Also as standard in SQL, this function resolves columns by position (not by name).
"""
return self.union(other)

@since(2.3)
def unionByName(self, other):
""" Returns a new :class:`DataFrame` containing union of rows in this and another frame.
Expand Down
14 changes: 14 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1852,6 +1852,20 @@ class Dataset[T] private[sql](
CombineUnions(Union(logicalPlan, other.logicalPlan))
}

/**
* Returns a new Dataset containing union of rows in this Dataset and another Dataset.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

say that this is an alias of union.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

* This is an alias for `union`.
*
* This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union (that does
* deduplication of elements), use this function followed by a [[distinct]].
*
* Also as standard in SQL, this function resolves columns by position (not by name).
*
* @group typedrel
* @since 2.0.0
*/
def unionAll(other: Dataset[T]): Dataset[T] = union(other)

/**
* Returns a new Dataset containing union of rows in this Dataset and another Dataset.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
unionDF.agg(avg('key), max('key), min('key), sum('key)),
Row(50.5, 100, 1, 25250) :: Nil
)

// unionAll is an alias of union
val unionAllDF = testData.unionAll(testData).unionAll(testData)
.unionAll(testData).unionAll(testData)

checkAnswer(unionDF, unionAllDF)
}

test("union should union DataFrames with UDTs (SPARK-13410)") {
Expand Down