Skip to content

Commit

Permalink
[SPARK-51258][SQL] Remove unnecessary inheritance from SQLConfHelper
Browse files Browse the repository at this point in the history
### 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 <beliefer@163.com>
Signed-off-by: beliefer <beliefer@163.com>
  • Loading branch information
beliefer committed Feb 22, 2025
1 parent 666f45d commit 30f4f4e
Show file tree
Hide file tree
Showing 4 changed files with 5 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.spark.sql.execution

import org.apache.spark.sql.ExperimentalMethods
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.classic.SparkSession
Expand All @@ -30,9 +29,9 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy
import org.apache.spark.sql.internal.SQLConf

class SparkPlanner(val session: SparkSession, val experimentalMethods: ExperimentalMethods)
extends SparkStrategies with SQLConfHelper {
extends SparkStrategies {

override def conf: SQLConf = session.sessionState.conf
def conf: SQLConf = session.sessionState.conf

def numPartitions: Int = conf.numShufflePartitions

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

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

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

import V1WritesUtils._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v2

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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.Path

import org.apache.spark.internal.LogKeys.{CHECKPOINT_LOCATION, CHECKPOINT_ROOT, CONFIG, PATH}
import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
Expand All @@ -38,7 +37,7 @@ import org.apache.spark.util.Utils
/**
* Replaces logical [[WriteToStreamStatement]] operator with an [[WriteToStream]] operator.
*/
object ResolveWriteToStream extends Rule[LogicalPlan] with SQLConfHelper {
object ResolveWriteToStream extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
case s: WriteToStreamStatement =>
val (resolvedCheckpointLocation, deleteCheckpointOnStop) = resolveCheckpointLocation(s)
Expand Down

0 comments on commit 30f4f4e

Please sign in to comment.