Skip to content

Commit 30f4f4e

Browse files
committed
[SPARK-51258][SQL] Remove unnecessary inheritance from SQLConfHelper
### What changes were proposed in this pull request? This PR proposes to remove unnecessary inheritance from `SQLConfHelper`. ### Why are the changes needed? 1. Some code doesn't need `SQLConfHelper`, we can remove it. 2. `SparkPlanner` no need to extends `SQLConfHelper`, because it can get the `SQLConf` from `SparkSession` directly. The other modification is related to the class Rule which already extends `SQLConfHelper`, so we should avoid the duplicated inheritance. ``` abstract class Rule[TreeType <: TreeNode[_]] extends SQLConfHelper with Logging { ... } ``` ### Does this PR introduce _any_ user-facing change? 'No'. Just update the inner code. ### How was this patch tested? GA. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #50005 from beliefer/SPARK-51258. Authored-by: beliefer <[email protected]> Signed-off-by: beliefer <[email protected]>
1 parent 666f45d commit 30f4f4e

File tree

4 files changed

+5
-9
lines changed

4 files changed

+5
-9
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.spark.sql.execution
1919

2020
import org.apache.spark.sql.ExperimentalMethods
21-
import org.apache.spark.sql.catalyst.SQLConfHelper
2221
import org.apache.spark.sql.catalyst.expressions._
2322
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2423
import org.apache.spark.sql.classic.SparkSession
@@ -30,9 +29,9 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy
3029
import org.apache.spark.sql.internal.SQLConf
3130

3231
class SparkPlanner(val session: SparkSession, val experimentalMethods: ExperimentalMethods)
33-
extends SparkStrategies with SQLConfHelper {
32+
extends SparkStrategies {
3433

35-
override def conf: SQLConf = session.sessionState.conf
34+
def conf: SQLConf = session.sessionState.conf
3635

3736
def numPartitions: Int = conf.numShufflePartitions
3837

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717

1818
package org.apache.spark.sql.execution.datasources
1919

20-
import org.apache.spark.sql.catalyst.SQLConfHelper
2120
import org.apache.spark.sql.catalyst.catalog.BucketSpec
2221
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
2322
import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeMap, AttributeSet, BitwiseAnd, Empty2Null, Expression, HiveHash, Literal, NamedExpression, Pmod, SortOrder}
@@ -65,7 +64,7 @@ trait V1WriteCommand extends DataWritingCommand {
6564
/**
6665
* A rule that plans v1 write for [[V1WriteCommand]].
6766
*/
68-
object V1Writes extends Rule[LogicalPlan] with SQLConfHelper {
67+
object V1Writes extends Rule[LogicalPlan] {
6968

7069
import V1WritesUtils._
7170

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v2
1818

1919
import org.apache.spark.internal.{Logging, MDC}
2020
import org.apache.spark.internal.LogKeys.CLASS_NAME
21-
import org.apache.spark.sql.catalyst.SQLConfHelper
2221
import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils
2322
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2423
import org.apache.spark.sql.catalyst.rules.Rule
@@ -32,7 +31,7 @@ import org.apache.spark.util.collection.Utils.sequenceToOption
3231
* and ordering reported by data sources to their catalyst counterparts. Then, annotates the plan
3332
* with the partitioning and ordering result.
3433
*/
35-
object V2ScanPartitioningAndOrdering extends Rule[LogicalPlan] with SQLConfHelper with Logging {
34+
object V2ScanPartitioningAndOrdering extends Rule[LogicalPlan] with Logging {
3635
override def apply(plan: LogicalPlan): LogicalPlan = {
3736
val scanRules = Seq[LogicalPlan => LogicalPlan] (partitioning, ordering)
3837

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.Path
2525

2626
import org.apache.spark.internal.LogKeys.{CHECKPOINT_LOCATION, CHECKPOINT_ROOT, CONFIG, PATH}
2727
import org.apache.spark.internal.MDC
28-
import org.apache.spark.sql.catalyst.SQLConfHelper
2928
import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
3029
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
3130
import org.apache.spark.sql.catalyst.rules.Rule
@@ -38,7 +37,7 @@ import org.apache.spark.util.Utils
3837
/**
3938
* Replaces logical [[WriteToStreamStatement]] operator with an [[WriteToStream]] operator.
4039
*/
41-
object ResolveWriteToStream extends Rule[LogicalPlan] with SQLConfHelper {
40+
object ResolveWriteToStream extends Rule[LogicalPlan] {
4241
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
4342
case s: WriteToStreamStatement =>
4443
val (resolvedCheckpointLocation, deleteCheckpointOnStop) = resolveCheckpointLocation(s)

0 commit comments

Comments
 (0)