This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new a77b70d  [SPARK-33788][SQL][3.1][3.0][2.4] Throw 
NoSuchPartitionsException from HiveExternalCatalog.dropPartitions()
a77b70d is described below

commit a77b70daf418ec75130d2ab7847aeb77aa602ecd
Author: Max Gekk <[email protected]>
AuthorDate: Wed Dec 16 23:30:47 2020 +0900

    [SPARK-33788][SQL][3.1][3.0][2.4] Throw NoSuchPartitionsException from 
HiveExternalCatalog.dropPartitions()
    
    ### What changes were proposed in this pull request?
    Throw `NoSuchPartitionsException` from `ALTER TABLE .. DROP TABLE` for not 
existing partitions of a table in V1 Hive external catalog.
    
    ### Why are the changes needed?
    The behaviour of Hive external catalog deviates from V1/V2 in-memory 
catalogs that throw `NoSuchPartitionsException`. To improve user experience 
with Spark SQL, it would be better to throw the same exception.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes, the command throws `NoSuchPartitionsException` instead of the general 
exception `AnalysisException`.
    
    ### How was this patch tested?
    By running new UT via:
    ```
    $ build/sbt -Phive -Phive-thriftserver "test:testOnly *HiveDDLSuite"
    ```
    
    Authored-by: Max Gekk <max.gekkgmail.com>
    Signed-off-by: HyukjinKwon <gurwls223apache.org>
    (cherry picked from commit 3dfdcf4f92ef5e739f15c22c93d673bb2233e617)
    Signed-off-by: Max Gekk <max.gekkgmail.com>
    
    Closes #30802 from MaxGekk/hive-drop-partition-exception-3.1.
    
    Authored-by: Max Gekk <[email protected]>
    Signed-off-by: HyukjinKwon <[email protected]>
---
 .../apache/spark/sql/hive/client/HiveClientImpl.scala  |  6 ++----
 .../apache/spark/sql/hive/execution/HiveDDLSuite.scala | 18 +++++++++++++++++-
 2 files changed, 19 insertions(+), 5 deletions(-)

diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
index 93ef5d4..f311836 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
@@ -49,7 +49,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.metrics.source.HiveCatalogMetrics
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, 
NoSuchPartitionException, PartitionsAlreadyExistException}
+import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, 
NoSuchPartitionException, NoSuchPartitionsException, 
PartitionsAlreadyExistException}
 import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
 import org.apache.spark.sql.catalyst.expressions.Expression
@@ -637,9 +637,7 @@ private[hive] class HiveClientImpl(
         // (b='1', c='1') and (b='1', c='2'), a partial spec of (b='1') will 
match both.
         val parts = client.getPartitions(hiveTable, s.asJava).asScala
         if (parts.isEmpty && !ignoreIfNotExists) {
-          throw new AnalysisException(
-            s"No partition is dropped. One partition spec '$s' does not exist 
in table '$table' " +
-            s"database '$db'")
+          throw new NoSuchPartitionsException(db, table, Seq(s))
         }
         parts.map(_.getValues)
       }.distinct
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index 4dd1488..765119d 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -28,7 +28,7 @@ import org.scalatest.BeforeAndAfterEach
 import org.apache.spark.SparkException
 import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode}
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, 
PartitionsAlreadyExistException, TableAlreadyExistsException}
+import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, 
NoSuchPartitionsException, PartitionsAlreadyExistException, 
TableAlreadyExistsException}
 import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.connector.catalog.CatalogManager
@@ -2750,4 +2750,20 @@ class HiveDDLSuite
       checkAnswer(sql("SHOW PARTITIONS t"), Seq(Row("id=1"), Row("id=2")))
     }
   }
+
+  test("SPARK-33788: partition not exists") {
+    withTable("t") {
+      sql(s"CREATE TABLE t (data string) PARTITIONED BY (id bigint)")
+      sql(s"ALTER TABLE t ADD PARTITION (id=1)")
+
+      val errMsg = intercept[NoSuchPartitionsException] {
+        sql(s"ALTER TABLE t DROP PARTITION (id=1), PARTITION (id=2)")
+      }.getMessage
+      assert(errMsg.contains("partitions not found in table"))
+
+      checkAnswer(sql("SHOW PARTITIONS t"), Seq(Row("id=1")))
+      sql(s"ALTER TABLE t DROP IF EXISTS PARTITION (id=1), PARTITION (id=2)")
+      checkAnswer(sql("SHOW PARTITIONS t"), Seq.empty)
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to