diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala index 04132c9f4c301..5632e46c04374 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala @@ -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 @@ -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 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala index 1d6c2a6f81124..280fe1068d814 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/V1Writes.scala @@ -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} @@ -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._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala index cfab28bbd15ed..7ee4374b45789 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala @@ -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 @@ -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) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala index 0d4ab9d147b8d..6c0a228ec2fca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala @@ -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 @@ -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)