Skip to content

Commit

Permalink
[SPARK-51258][SQL] Remove unnecessary inheritance from SQLConfHelper
Browse files Browse the repository at this point in the history
  • Loading branch information
beliefer committed Feb 19, 2025
1 parent c36916f commit 25cf58e
Show file tree
Hide file tree
Showing 6 changed files with 8 additions and 17 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import scala.collection.mutable

import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
Expand All @@ -36,7 +35,7 @@ import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
import org.apache.spark.sql.types.{ArrayType, DataType, DecimalType, IntegralType, MapType, StructType, UserDefinedType}

object TableOutputResolver extends SQLConfHelper with Logging {
object TableOutputResolver extends Logging {

def resolveVariableOutputColumns(
expected: Seq[VariableReference],
Expand Down
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 @@ -27,14 +26,11 @@ import org.apache.spark.sql.execution.adaptive.LogicalQueryStageStrategy
import org.apache.spark.sql.execution.command.v2.V2CommandStrategy
import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileSourceStrategy}
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 numPartitions: Int = conf.numShufflePartitions
def numPartitions: Int = session.sessionState.conf.numShufflePartitions

override def strategies: Seq[Strategy] =
experimentalMethods.extraStrategies ++
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,13 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark.SparkException
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.{SQLConfHelper, TableIdentifier}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, GlobalTempView, LocalTempView, SchemaEvolution, SchemaUnsupported, ViewSchemaMode, ViewType}
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, TemporaryViewRelation}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, SubqueryExpression, VariableReference}
import org.apache.spark.sql.catalyst.plans.logical.{AnalysisOnlyCommand, CTEInChildren, CTERelationDef, LogicalPlan, Project, View, WithCTE}
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.classic.ClassicConversions.castToImpl
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.types.{MetadataBuilder, StructType}
Expand Down Expand Up @@ -409,7 +408,7 @@ case class ShowViewsCommand(
}
}

object ViewHelper extends SQLConfHelper with Logging {
object ViewHelper extends Logging {

private val configPrefixDenyList = Seq(
SQLConf.MAX_NESTED_VIEW_DEPTH.key,
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 25cf58e

Please sign in to comment.