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

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new eff46ea77e9 [SPARK-45340][SQL] Remove the SQL config 
`spark.sql.hive.verifyPartitionPath`
eff46ea77e9 is described below

commit eff46ea77e9bebe7777f3076277bef1e086833dd
Author: Max Gekk <max.g...@gmail.com>
AuthorDate: Wed Sep 27 08:28:45 2023 +0300

    [SPARK-45340][SQL] Remove the SQL config 
`spark.sql.hive.verifyPartitionPath`
    
    ### What changes were proposed in this pull request?
    In the PR, I propose to remove already deprecated SQL config 
`spark.sql.hive.verifyPartitionPath`, and the code under the config. The config 
has been deprecated since Spark 3.0.
    
    ### Why are the changes needed?
    To improve code maintainability by remove unused code.
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    By running the modified test suite:
    ```
    $ build/sbt "test:testOnly *SQLConfSuite"
    $ build/sbt "test:testOnly *QueryPartitionSuite"
    ```
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #43130 from MaxGekk/remove-verifyPartitionPath.
    
    Authored-by: Max Gekk <max.g...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../org/apache/spark/sql/internal/SQLConf.scala    | 17 ++-------
 .../apache/spark/sql/internal/SQLConfSuite.scala   |  4 +--
 .../org/apache/spark/sql/hive/TableReader.scala    | 41 +---------------------
 .../spark/sql/hive/QueryPartitionSuite.scala       | 12 ++-----
 4 files changed, 8 insertions(+), 66 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 43eb0756d8d..aeef531dbcd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -34,7 +34,6 @@ import org.apache.hadoop.fs.Path
 import org.apache.spark.{ErrorMessageFormat, SparkConf, SparkContext, 
TaskContext}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
-import org.apache.spark.internal.config.{IGNORE_MISSING_FILES => 
SPARK_IGNORE_MISSING_FILES}
 import org.apache.spark.network.util.ByteUnit
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver}
@@ -1261,14 +1260,6 @@ object SQLConf {
     .booleanConf
     .createWithDefault(false)
 
-  val HIVE_VERIFY_PARTITION_PATH = 
buildConf("spark.sql.hive.verifyPartitionPath")
-    .doc("When true, check all the partition paths under the table\'s root 
directory " +
-         "when reading data stored in HDFS. This configuration will be 
deprecated in the future " +
-         s"releases and replaced by ${SPARK_IGNORE_MISSING_FILES.key}.")
-    .version("1.4.0")
-    .booleanConf
-    .createWithDefault(false)
-
   val HIVE_METASTORE_DROP_PARTITION_BY_NAME =
     buildConf("spark.sql.hive.dropPartitionByName.enabled")
       .doc("When true, Spark will get partition name rather than partition 
object " +
@@ -4472,8 +4463,6 @@ object SQLConf {
         PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key, "2.4",
         "The config allows to switch to the behaviour before Spark 2.4 " +
           "and will be removed in the future releases."),
-      DeprecatedConfig(HIVE_VERIFY_PARTITION_PATH.key, "3.0",
-        s"This config is replaced by '${SPARK_IGNORE_MISSING_FILES.key}'."),
       DeprecatedConfig(ARROW_EXECUTION_ENABLED.key, "3.0",
         s"Use '${ARROW_PYSPARK_EXECUTION_ENABLED.key}' instead of it."),
       DeprecatedConfig(ARROW_FALLBACK_ENABLED.key, "3.0",
@@ -4552,7 +4541,9 @@ object SQLConf {
       RemovedConfig("spark.sql.ansi.strictIndexOperator", "3.4.0", "true",
         "This was an internal configuration. It is not needed anymore since 
Spark SQL always " +
           "returns null when getting a map value with a non-existing key. See 
SPARK-40066 " +
-          "for more details.")
+          "for more details."),
+      RemovedConfig("spark.sql.hive.verifyPartitionPath", "4.0.0", "false",
+        s"This config was replaced by '${IGNORE_MISSING_FILES.key}'.")
     )
 
     Map(configs.map { cfg => cfg.key -> cfg } : _*)
@@ -4766,8 +4757,6 @@ class SQLConf extends Serializable with Logging with 
SqlApiConf {
 
   def isOrcSchemaMergingEnabled: Boolean = getConf(ORC_SCHEMA_MERGING_ENABLED)
 
-  def verifyPartitionPath: Boolean = getConf(HIVE_VERIFY_PARTITION_PATH)
-
   def metastoreDropPartitionsByName: Boolean = 
getConf(HIVE_METASTORE_DROP_PARTITION_BY_NAME)
 
   def metastorePartitionPruning: Boolean = 
getConf(HIVE_METASTORE_PARTITION_PRUNING)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
index 0a0bee2eabd..822c0642f2b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
@@ -422,9 +422,9 @@ class SQLConfSuite extends QueryTest with 
SharedSparkSession {
         e.getMessage.getFormattedMessage.contains(config)))
     }
 
-    val config1 = SQLConf.HIVE_VERIFY_PARTITION_PATH.key
+    val config1 = SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key
     withLogAppender(logAppender) {
-      spark.conf.set(config1, true)
+      spark.conf.set(config1, 1)
     }
     check(config1)
 
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index 5bb982624b0..a61040b2fab 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -160,46 +160,7 @@ class HadoopTableReader(
   def makeRDDForPartitionedTable(
       partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]],
       filterOpt: Option[PathFilter]): RDD[InternalRow] = {
-
-    // SPARK-5068:get FileStatus and do the filtering locally when the path is 
not exists
-    def verifyPartitionPath(
-        partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]):
-        Map[HivePartition, Class[_ <: Deserializer]] = {
-      if (!conf.verifyPartitionPath) {
-        partitionToDeserializer
-      } else {
-        val existPathSet = collection.mutable.Set[String]()
-        val pathPatternSet = collection.mutable.Set[String]()
-        partitionToDeserializer.filter {
-          case (partition, partDeserializer) =>
-            def updateExistPathSetByPathPattern(pathPatternStr: String): Unit 
= {
-              val pathPattern = new Path(pathPatternStr)
-              val fs = pathPattern.getFileSystem(hadoopConf)
-              val matches = fs.globStatus(pathPattern)
-              matches.foreach(fileStatus => existPathSet += 
fileStatus.getPath.toString)
-            }
-            // convert  /demo/data/year/month/day  to  /demo/data/*/*/*/
-            def getPathPatternByPath(parNum: Int, tempPath: Path): String = {
-              var path = tempPath
-              for (i <- (1 to parNum)) path = path.getParent
-              val tails = (1 to parNum).map(_ => "*").mkString("/", "/", "/")
-              path.toString + tails
-            }
-
-            val partPath = partition.getDataLocation
-            val partNum = 
Utilities.getPartitionDesc(partition).getPartSpec.size()
-            val pathPatternStr = getPathPatternByPath(partNum, partPath)
-            if (!pathPatternSet.contains(pathPatternStr)) {
-              pathPatternSet += pathPatternStr
-              updateExistPathSetByPathPattern(pathPatternStr)
-            }
-            existPathSet.contains(partPath.toString)
-        }
-      }
-    }
-
-    val hivePartitionRDDs = verifyPartitionPath(partitionToDeserializer)
-      .map { case (partition, partDeserializer) =>
+    val hivePartitionRDDs = partitionToDeserializer.map { case (partition, 
partDeserializer) =>
       val partDesc = Utilities.getPartitionDescFromTableDesc(tableDesc, 
partition, true)
       val partPath = partition.getDataLocation
       val inputPathStr = applyFilterIfNeeded(partPath, filterOpt)
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
index f4fb18119fa..c1be1cee005 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
@@ -65,16 +65,8 @@ class QueryPartitionSuite extends QueryTest with 
SQLTestUtils with TestHiveSingl
     }
   }
 
-  test("SPARK-5068: query data when path doesn't exist") {
-    withSQLConf(SQLConf.HIVE_VERIFY_PARTITION_PATH.key -> "true") {
-      queryWhenPathNotExist()
-    }
-  }
-
-  test("Replace spark.sql.hive.verifyPartitionPath by 
spark.files.ignoreMissingFiles") {
-    withSQLConf(
-      SQLConf.HIVE_VERIFY_PARTITION_PATH.key -> "false",
-      SQLConf.IGNORE_MISSING_FILES.key -> "true") {
+  test("Replace spark.sql.hive.verifyPartitionPath by 
spark.sql.files.ignoreMissingFiles") {
+    withSQLConf(SQLConf.IGNORE_MISSING_FILES.key -> "true") {
       queryWhenPathNotExist()
     }
   }


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

Reply via email to