Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 6 additions & 5 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -440,6 +440,12 @@
],
"sqlState" : "46110"
},
"CANNOT_MODIFY_STATIC_CONFIG" : {
"message" : [
"Cannot modify the value of the static Spark config: <key>."
],
"sqlState" : "46110"
},
"CANNOT_PARSE_DECIMAL" : {
"message" : [
"Cannot parse decimal. Please ensure that the input is a valid number with optional decimal point or comma separators."
Expand Down Expand Up @@ -9144,11 +9150,6 @@
"Failed to get block <blockId>, which is not a shuffle block"
]
},
"_LEGACY_ERROR_TEMP_3050" : {
"message" : [
"Cannot modify the value of a static config: <k>"
]
},
"_LEGACY_ERROR_TEMP_3052" : {
"message" : [
"Unexpected resolved action: <other>"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,18 @@ private[sql] trait CompilationErrors extends DataTypeErrorsBase {
errorClass = "SPECIFY_CLUSTER_BY_WITH_BUCKETING_IS_NOT_ALLOWED",
messageParameters = Map.empty)
}

def cannotModifyValueOfStaticConfigError(key: String): Throwable = {
new AnalysisException(
errorClass = "CANNOT_MODIFY_STATIC_CONFIG",
messageParameters = Map("key" -> toSQLConf(key)))
Copy link
Member Author

@pan3793 pan3793 Oct 13, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@LuciferYang I updated the code to remove the doc link since <docroot>/sql-migration-guide.html#ddl-statements does not provide useful info for this error

}

def cannotModifyValueOfSparkConfigError(key: String, docroot: String): Throwable = {
new AnalysisException(
errorClass = "CANNOT_MODIFY_CONFIG",
messageParameters = Map("key" -> toSQLConf(key), "docroot" -> docroot))
}
}

private[sql] object CompilationErrors extends CompilationErrors
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.sql.catalyst

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.errors.CompilationErrors
import org.apache.spark.sql.internal.SQLConf

/**
Expand Down Expand Up @@ -47,9 +47,7 @@ trait SQLConfHelper {
}
keys.lazyZip(values).foreach { (k, v) =>
if (SQLConf.isStaticConfigKey(k)) {
throw new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_3050",
messageParameters = Map("k" -> k))
throw CompilationErrors.cannotModifyValueOfStaticConfigError(k)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is it necessary to direct users to refer to <docroot>/sql-migration-guide.html#ddl-statements here?

Copy link
Member Author

@pan3793 pan3793 Oct 13, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@LuciferYang I think it might be because of

- Spark 2.4 and below: the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. In 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`.

image

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I see your point, here spark.sql.legacy.setCommandRejectsSparkCoreConfs does not take effect, so the docs do not help

}
conf.setConfString(k, v)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3475,19 +3475,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
"config" -> SQLConf.DATAFRAME_PIVOT_MAX_VALUES.key))
}

def cannotModifyValueOfStaticConfigError(key: String): Throwable = {
new AnalysisException(
errorClass = "CANNOT_MODIFY_CONFIG",
messageParameters = Map("key" -> toSQLConf(key), "docroot" -> SPARK_DOC_ROOT)
)
}

def cannotModifyValueOfSparkConfigError(key: String, docroot: String): Throwable = {
new AnalysisException(
errorClass = "CANNOT_MODIFY_CONFIG",
messageParameters = Map("key" -> toSQLConf(key), "docroot" -> docroot))
}

def commandExecutionInRunnerUnsupportedError(runner: String): Throwable = {
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1327",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ import java.util.UUID

import org.scalatest.Assertions.fail

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
import org.apache.spark.sql.connect.{DataFrame, SparkSession, SQLImplicits}
import org.apache.spark.sql.errors.CompilationErrors
import org.apache.spark.util.{SparkErrorUtils, SparkFileUtils}

trait SQLHelper {
Expand Down Expand Up @@ -59,11 +59,8 @@ trait SQLHelper {
if (spark.conf.isModifiable(k)) {
spark.conf.set(k, v)
} else {
throw new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_3050",
messageParameters = Map("k" -> k))
throw CompilationErrors.cannotModifyValueOfStaticConfigError(k)
}

}
try f
finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -236,8 +236,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession {
// static sql configs
checkError(
exception = intercept[AnalysisException](sql(s"RESET ${StaticSQLConf.WAREHOUSE_PATH.key}")),
condition = "CANNOT_MODIFY_CONFIG",
parameters = Map("key" -> "\"spark.sql.warehouse.dir\"", "docroot" -> SPARK_DOC_ROOT))
condition = "CANNOT_MODIFY_STATIC_CONFIG",
parameters = Map("key" -> "\"spark.sql.warehouse.dir\""))

}

Expand Down Expand Up @@ -348,13 +348,13 @@ class SQLConfSuite extends QueryTest with SharedSparkSession {
test("cannot set/unset static SQL conf") {
checkError(
exception = intercept[AnalysisException](sql(s"SET ${GLOBAL_TEMP_DATABASE.key}=10")),
condition = "CANNOT_MODIFY_CONFIG",
parameters = Map("key" -> "\"spark.sql.globalTempDatabase\"", "docroot" -> SPARK_DOC_ROOT)
condition = "CANNOT_MODIFY_STATIC_CONFIG",
parameters = Map("key" -> "\"spark.sql.globalTempDatabase\"")
)
checkError(
exception = intercept[AnalysisException](spark.conf.unset(GLOBAL_TEMP_DATABASE.key)),
condition = "CANNOT_MODIFY_CONFIG",
parameters = Map("key" -> "\"spark.sql.globalTempDatabase\"", "docroot" -> SPARK_DOC_ROOT)
condition = "CANNOT_MODIFY_STATIC_CONFIG",
parameters = Map("key" -> "\"spark.sql.globalTempDatabase\"")
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1063,7 +1063,7 @@ class SingleSessionSuite extends HiveThriftServer2TestBase {
statement.executeQuery("SET spark.sql.hive.thriftServer.singleSession=false")
}.getMessage
assert(e.contains(
"CANNOT_MODIFY_CONFIG"))
"CANNOT_MODIFY_STATIC_CONFIG"))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import java.util.{Locale, Set}

import org.apache.hadoop.fs.{FileSystem, Path}

import org.apache.spark.{SPARK_DOC_ROOT, SparkException, TestUtils}
import org.apache.spark.{SparkException, TestUtils}
import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.TableIdentifier
Expand Down Expand Up @@ -2464,9 +2464,8 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi
"spark.sql.hive.metastore.barrierPrefixes").foreach { key =>
checkError(
exception = intercept[AnalysisException](sql(s"set $key=abc")),
condition = "CANNOT_MODIFY_CONFIG",
parameters = Map(
"key" -> toSQLConf(key), "docroot" -> SPARK_DOC_ROOT)
condition = "CANNOT_MODIFY_STATIC_CONFIG",
parameters = Map("key" -> toSQLConf(key))
)
}
}
Expand Down