Repository: spark
Updated Branches:
  refs/heads/master c42ef9533 -> 8cd9cdf17


[SPARK-21333][DOCS] Removed invalid joinTypes from javadoc of Dataset#joinWith

## What changes were proposed in this pull request?

Two invalid join types were mistakenly listed in the javadoc for joinWith, in 
the Dataset class. I presume these were copied from the javadoc of join, but 
since joinWith returns a Dataset\<Tuple2\>, left_semi and left_anti are 
invalid, as they only return values from one of the datasets, instead of from 
both

## How was this patch tested?

I ran the following code :
```
public static void main(String[] args) {
        SparkSession spark = new SparkSession(new SparkContext("local[*]", 
"Test"));
        Dataset<Row> one = spark.createDataFrame(Arrays.asList(new Bean(1), new 
Bean(2), new Bean(3), new Bean(4), new Bean(5)), Bean.class);
        Dataset<Row> two = spark.createDataFrame(Arrays.asList(new Bean(4), new 
Bean(5), new Bean(6), new Bean(7), new Bean(8), new Bean(9)), Bean.class);

        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"inner").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"cross").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"outer").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"full").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"full_outer").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"left").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"left_outer").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"right").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"right_outer").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"left_semi").show();} catch (Exception e) {e.printStackTrace();}
        try {two.joinWith(one, one.col("x").equalTo(two.col("x")), 
"left_anti").show();} catch (Exception e) {e.printStackTrace();}
}
```
which tests all the different join types, and the last two (left_semi and 
left_anti) threw exceptions. The same code using join instead of joinWith did 
fine. The Bean class was just a java bean with a single int field, x.

Author: Corey Woodfield <coreywoodfi...@gmail.com>

Closes #18462 from coreywoodfield/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8cd9cdf1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8cd9cdf1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8cd9cdf1

Branch: refs/heads/master
Commit: 8cd9cdf17a7a4ad6f2eecd7c4b388ca363c20982
Parents: c42ef95
Author: Corey Woodfield <coreywoodfi...@gmail.com>
Authored: Wed Jul 19 15:21:38 2017 -0700
Committer: gatorsmile <gatorsm...@gmail.com>
Committed: Wed Jul 19 15:21:38 2017 -0700

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/sql/Dataset.scala   |  6 +++++-
 .../scala/org/apache/spark/sql/DatasetSuite.scala   | 16 ++++++++++++++++
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/8cd9cdf1/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index b825b6c..71ab0dd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -980,7 +980,7 @@ class Dataset[T] private[sql](
    * @param condition Join expression.
    * @param joinType Type of join to perform. Default `inner`. Must be one of:
    *                 `inner`, `cross`, `outer`, `full`, `full_outer`, `left`, 
`left_outer`,
-   *                 `right`, `right_outer`, `left_semi`, `left_anti`.
+   *                 `right`, `right_outer`.
    *
    * @group typedrel
    * @since 1.6.0
@@ -997,6 +997,10 @@ class Dataset[T] private[sql](
         JoinType(joinType),
         Some(condition.expr))).analyzed.asInstanceOf[Join]
 
+    if (joined.joinType == LeftSemi || joined.joinType == LeftAnti) {
+      throw new AnalysisException("Invalid join type in joinWith: " + 
joined.joinType.sql)
+    }
+
     // For both join side, combine all outputs into a single column and alias 
it with "_1" or "_2",
     // to match the schema for the encoder of the join result.
     // Note that we do this before joining them, to enable the join operator 
to return null for one

http://git-wip-us.apache.org/repos/asf/spark/blob/8cd9cdf1/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 69d110e..73098cd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -21,6 +21,7 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput}
 import java.sql.{Date, Timestamp}
 
 import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder}
+import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi}
 import org.apache.spark.sql.catalyst.util.sideBySide
 import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec}
 import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, 
ShuffleExchange}
@@ -400,6 +401,21 @@ class DatasetSuite extends QueryTest with SharedSQLContext 
{
       ((("b", 2), ("b", 2)), ("b", 2)))
   }
 
+  test("joinWith join types") {
+    val ds1 = Seq(1, 2, 3).toDS().as("a")
+    val ds2 = Seq(1, 2).toDS().as("b")
+
+    val e1 = intercept[AnalysisException] {
+      ds1.joinWith(ds2, $"a.value" === $"b.value", "left_semi")
+    }.getMessage
+    assert(e1.contains("Invalid join type in joinWith: " + LeftSemi.sql))
+
+    val e2 = intercept[AnalysisException] {
+      ds1.joinWith(ds2, $"a.value" === $"b.value", "left_anti")
+    }.getMessage
+    assert(e2.contains("Invalid join type in joinWith: " + LeftAnti.sql))
+  }
+
   test("groupBy function, keys") {
     val ds = Seq(("a", 1), ("b", 1)).toDS()
     val grouped = ds.groupByKey(v => (1, v._2))


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to