diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 1d3cc88c9fac7..ee5d37cebf2f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.Resolver @@ -613,7 +614,7 @@ case class AlterTableRecoverPartitionsCommand( val hadoopConf = spark.sessionState.newHadoopConf() val fs = root.getFileSystem(hadoopConf) - val threshold = spark.conf.get("spark.rdd.parallelListingThreshold", "10").toInt + val threshold = spark.sparkContext.conf.get(RDD_PARALLEL_LISTING_THRESHOLD) val pathFilter = getPathFilter(hadoopConf) val evalPool = ThreadUtils.newForkJoinPool("AlterTableRecoverPartitionsCommand", 8) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index ba39dcf77b470..4859bde525612 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach import org.apache.spark.internal.config +import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} @@ -1122,13 +1123,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } test("alter table: recover partitions (sequential)") { - withSQLConf("spark.rdd.parallelListingThreshold" -> "10") { + withSQLConf(RDD_PARALLEL_LISTING_THRESHOLD.key -> "10") { testRecoverPartitions() } } test("alter table: recover partition (parallel)") { - withSQLConf("spark.rdd.parallelListingThreshold" -> "0") { + withSQLConf(RDD_PARALLEL_LISTING_THRESHOLD.key -> "0") { testRecoverPartitions() } }