Skip to content
Open
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
1 change: 1 addition & 0 deletions docs/sql-migration-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ license: |
## Upgrading from Spark SQL 4.1 to 4.2

- Since Spark 4.2, Spark enables order-independent checksums for shuffle outputs by default to detect data inconsistencies during indeterminate shuffle stage retries. If a checksum mismatch is detected, Spark rolls back and re-executes all succeeding stages that depend on the shuffle output. If rolling back is not possible for some succeeding stages, the job will fail. To restore the previous behavior, set `spark.sql.shuffle.orderIndependentChecksum.enabled` and `spark.sql.shuffle.orderIndependentChecksum.enableFullRetryOnMismatch` to `false`.
- Since Spark 4.2, Spark properly enforces NOT NULL constraints when inserting data into V1 file-based data source tables (e.g., Parquet, ORC, JSON). Previously, null values were silently accepted into NOT NULL columns. To restore the previous behavior, set `spark.sql.legacy.allowNullInsertForFileSourceTables` to `true`.
- Since Spark 4.2, support for Derby JDBC datasource is deprecated.

## Upgrading from Spark SQL 4.0 to 4.1
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4743,6 +4743,17 @@ object SQLConf {
.enumConf(StoreAssignmentPolicy)
.createWithDefault(StoreAssignmentPolicy.ANSI)

val LEGACY_NO_NULL_CHECK_FOR_FILE_SOURCE_INSERT =
buildConf("spark.sql.legacy.allowNullInsertForFileSourceTables")
.internal()
.doc("When true (legacy behavior), Spark does not enforce NOT NULL constraints " +
"when inserting data into file-based data source tables (e.g., Parquet, ORC, JSON). " +
"When false, Spark properly enforces NOT NULL constraints for file-based tables, " +
"consistent with the behavior for other data sources and V2 catalog tables.")
.version("4.2.0")
.booleanConf
.createWithDefault(false)

val ANSI_ENABLED = buildConf(SqlApiConfHelper.ANSI_ENABLED_KEY)
.doc("When true, Spark SQL uses an ANSI compliant dialect instead of being Hive compliant. " +
"For example, Spark will throw an exception at runtime instead of returning null results " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.CommandExecutionMode
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType}
import org.apache.spark.util.ArrayImplicits._

/**
Expand Down Expand Up @@ -107,8 +107,17 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo
table.copy(schema = new StructType(), partitionColumnNames = Nil)

case _ =>
// Merge nullability from the user-specified schema into the resolved schema.
// DataSource.resolveRelation() calls dataSchema.asNullable which strips NOT NULL
// constraints. We restore nullability from the original user schema while keeping
// the resolved data types (which may include CharVarchar normalization, metadata, etc.)
val resolvedSchema = if (table.schema.nonEmpty) {
restoreNullability(dataSource.schema, table.schema)
} else {
dataSource.schema
}
table.copy(
schema = dataSource.schema,
schema = resolvedSchema,
partitionColumnNames = partitionColumnNames,
// If metastore partition management for file source tables is enabled, we start off with
// partition provider hive, but no partitions in the metastore. The user has to call
Expand All @@ -122,6 +131,39 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo

Seq.empty[Row]
}

/**
* Recursively restores nullability from the original user-specified schema into
* the resolved schema. The resolved schema's data types are preserved (they may
* contain CharVarchar normalization, metadata, etc.), but nullability flags
* (top-level and nested) are taken from the original schema.
*/
private def restoreNullability(resolved: StructType, original: StructType): StructType = {
val originalFields = original.fields.map(f => f.name -> f).toMap
StructType(resolved.fields.map { resolvedField =>
originalFields.get(resolvedField.name) match {
case Some(origField) =>
resolvedField.copy(
nullable = origField.nullable,
dataType = restoreDataTypeNullability(resolvedField.dataType, origField.dataType))
case None => resolvedField
}
})
}

private def restoreDataTypeNullability(resolved: DataType, original: DataType): DataType = {
(resolved, original) match {
case (r: StructType, o: StructType) => restoreNullability(r, o)
case (ArrayType(rElem, _), ArrayType(oElem, oNull)) =>
ArrayType(restoreDataTypeNullability(rElem, oElem), oNull)
case (MapType(rKey, rVal, _), MapType(oKey, oVal, oValNull)) =>
MapType(
restoreDataTypeNullability(rKey, oKey),
restoreDataTypeNullability(rVal, oVal),
oValNull)
case _ => resolved
}
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.datasources.{CreateTable => CreateTableV1}
import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.InsertableRelation
import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType}
import org.apache.spark.sql.types.{ArrayType, DataType, MapType, MetadataBuilder, StructField, StructType}
import org.apache.spark.sql.util.PartitioningUtils.normalizePartitionSpec
import org.apache.spark.sql.util.SchemaUtils
import org.apache.spark.util.ArrayImplicits._
Expand Down Expand Up @@ -470,7 +470,29 @@ object PreprocessTableInsertion extends ResolveInsertionBase {
insert.partitionSpec, partColNames, tblName, conf.resolver)

val staticPartCols = normalizedPartSpec.filter(_._2.isDefined).keySet
val expectedColumns = insert.table.output.filterNot(a => staticPartCols.contains(a.name))
val expectedColumns = {
val cols = insert.table.output.filterNot(a => staticPartCols.contains(a.name))
// When the legacy config is disabled, restore the original nullability from the
// catalog table schema. HadoopFsRelation forces dataSchema.asNullable for safe reads,
// which strips NOT NULL constraints (both top-level and nested) from the
// LogicalRelation output. We restore nullability so that AssertNotNull checks are
// properly injected.
if (!conf.getConf(SQLConf.LEGACY_NO_NULL_CHECK_FOR_FILE_SOURCE_INSERT)) {
catalogTable.map { ct =>
val catalogFields = ct.schema.fields.map(f => f.name -> f).toMap
cols.map { col =>
catalogFields.get(col.name) match {
case Some(field) =>
col.withNullability(field.nullable)
.withDataType(restoreDataTypeNullability(col.dataType, field.dataType))
case None => col
}
}
}.getOrElse(cols)
} else {
cols
}
}

val partitionsTrackedByCatalog = catalogTable.isDefined &&
catalogTable.get.partitionColumnNames.nonEmpty &&
Expand Down Expand Up @@ -546,6 +568,34 @@ object PreprocessTableInsertion extends ResolveInsertionBase {
case _ => i
}
}

/**
* Recursively restores nullability flags from the original data type into the resolved
* data type, keeping the resolved type structure intact.
*/
private def restoreDataTypeNullability(resolved: DataType, original: DataType): DataType = {
(resolved, original) match {
case (r: StructType, o: StructType) =>
val origFields = o.fields.map(f => f.name -> f).toMap
StructType(r.fields.map { rf =>
origFields.get(rf.name) match {
case Some(of) =>
rf.copy(
nullable = of.nullable,
dataType = restoreDataTypeNullability(rf.dataType, of.dataType))
case None => rf
}
})
case (ArrayType(rElem, _), ArrayType(oElem, oNull)) =>
ArrayType(restoreDataTypeNullability(rElem, oElem), oNull)
case (MapType(rKey, rVal, _), MapType(oKey, oVal, oValNull)) =>
MapType(
restoreDataTypeNullability(rKey, oKey),
restoreDataTypeNullability(rVal, oVal),
oValNull)
case _ => resolved
}
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ trait ShowCreateTableSuiteBase extends command.ShowCreateTableSuiteBase
val showDDL = getShowCreateDDL(t)
assert(showDDL === Array(
s"CREATE TABLE $fullName (",
"a BIGINT,",
"a BIGINT NOT NULL,",
"b BIGINT DEFAULT 42,",
"c STRING COLLATE UTF8_BINARY DEFAULT 'abc, \"def\"' COMMENT 'comment')",
"USING parquet",
Expand Down
170 changes: 170 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2851,6 +2851,176 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
}
}

test("SPARK-55716: V1 INSERT rejects null into NOT NULL column for file sources") {
Seq("parquet", "orc", "json").foreach { format =>
withTable("t") {
sql(s"CREATE TABLE t(i INT NOT NULL, s STRING NOT NULL) USING $format")
// V1 DataSource writes now enforce NOT NULL constraints via AssertNotNull
val e1 = intercept[SparkRuntimeException] {
sql("INSERT INTO t VALUES(null, 'a')")
}
assert(e1.getCondition === "NOT_NULL_ASSERT_VIOLATION")
val e2 = intercept[SparkRuntimeException] {
sql("INSERT INTO t VALUES(1, null)")
}
assert(e2.getCondition === "NOT_NULL_ASSERT_VIOLATION")
// Valid insert should succeed
sql("INSERT INTO t VALUES(1, 'a')")
checkAnswer(spark.table("t"), Seq(Row(1, "a")))
}
}
}

test("SPARK-55716: V1 INSERT NOT NULL enforcement respects storeAssignmentPolicy") {
Seq("parquet", "orc").foreach { format =>
// ANSI mode (default): rejects null
withSQLConf(
SQLConf.STORE_ASSIGNMENT_POLICY.key -> SQLConf.StoreAssignmentPolicy.ANSI.toString) {
withTable("t") {
sql(s"CREATE TABLE t(i INT NOT NULL) USING $format")
val e = intercept[SparkRuntimeException] {
sql("INSERT INTO t VALUES(null)")
}
assert(e.getCondition === "NOT_NULL_ASSERT_VIOLATION")
}
}
// STRICT mode: also rejects null (fails at analysis with type mismatch)
withSQLConf(
SQLConf.STORE_ASSIGNMENT_POLICY.key -> SQLConf.StoreAssignmentPolicy.STRICT.toString) {
withTable("t") {
sql(s"CREATE TABLE t(i INT NOT NULL) USING $format")
intercept[AnalysisException] {
sql("INSERT INTO t VALUES(null)")
}
}
}
// LEGACY mode: allows null (no AssertNotNull injected)
withSQLConf(
SQLConf.STORE_ASSIGNMENT_POLICY.key -> SQLConf.StoreAssignmentPolicy.LEGACY.toString) {
withTable("t") {
sql(s"CREATE TABLE t(i INT NOT NULL) USING $format")
sql("INSERT INTO t VALUES(null)")
checkAnswer(spark.table("t"), Seq(Row(null)))
}
}
// Legacy config: allows null even in ANSI mode
withSQLConf(
SQLConf.LEGACY_NO_NULL_CHECK_FOR_FILE_SOURCE_INSERT.key -> "true") {
withTable("t") {
sql(s"CREATE TABLE t(i INT NOT NULL) USING $format")
sql("INSERT INTO t VALUES(null)")
checkAnswer(spark.table("t"), Seq(Row(null)))
}
}
}
}

test("SPARK-55716: V1 INSERT rejects null with V2 file source path") {
Seq("parquet", "orc").foreach { format =>
withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") {
withTable("t") {
sql(s"CREATE TABLE t(i INT NOT NULL, s STRING NOT NULL) USING $format")
val e = intercept[SparkRuntimeException] {
sql("INSERT INTO t VALUES(null, 'a')")
}
assert(e.getCondition === "NOT_NULL_ASSERT_VIOLATION")
}
}
}
}

test("SPARK-55716: V1 INSERT rejects null array element for NOT NULL element type") {
Seq("parquet", "orc").foreach { format =>
withTable("t") {
val schema = new StructType()
.add("a", ArrayType(IntegerType, containsNull = false))
spark.sessionState.catalog.createTable(
CatalogTable(
identifier = TableIdentifier("t"),
tableType = CatalogTableType.MANAGED,
storage = CatalogStorageFormat.empty,
schema = schema,
provider = Some(format)),
ignoreIfExists = false)
val e = intercept[SparkRuntimeException] {
sql("INSERT INTO t SELECT array(1, null, 3)")
}
assert(e.getCondition === "NOT_NULL_ASSERT_VIOLATION")
// Valid insert should succeed
sql("INSERT INTO t SELECT array(1, 2, 3)")
checkAnswer(spark.table("t"), Seq(Row(Seq(1, 2, 3))))
}
}
}

test("SPARK-55716: V1 INSERT rejects null struct field for NOT NULL field") {
Seq("parquet", "orc").foreach { format =>
withTable("t") {
val schema = new StructType()
.add("s", new StructType()
.add("x", IntegerType, nullable = false)
.add("y", StringType, nullable = false))
spark.sessionState.catalog.createTable(
CatalogTable(
identifier = TableIdentifier("t"),
tableType = CatalogTableType.MANAGED,
storage = CatalogStorageFormat.empty,
schema = schema,
provider = Some(format)),
ignoreIfExists = false)
val e = intercept[SparkRuntimeException] {
sql("INSERT INTO t SELECT named_struct('x', null, 'y', 'hello')")
}
assert(e.getCondition === "NOT_NULL_ASSERT_VIOLATION")
// Valid insert should succeed
sql("INSERT INTO t SELECT named_struct('x', 1, 'y', 'hello')")
checkAnswer(spark.table("t"), Seq(Row(Row(1, "hello"))))
}
}
}

test("SPARK-55716: V1 INSERT rejects null map value for NOT NULL value type") {
Seq("parquet", "orc").foreach { format =>
withTable("t") {
val schema = new StructType()
.add("m", MapType(StringType, IntegerType, valueContainsNull = false))
spark.sessionState.catalog.createTable(
CatalogTable(
identifier = TableIdentifier("t"),
tableType = CatalogTableType.MANAGED,
storage = CatalogStorageFormat.empty,
schema = schema,
provider = Some(format)),
ignoreIfExists = false)
val e = intercept[SparkRuntimeException] {
sql("INSERT INTO t SELECT map('a', 1, 'b', null)")
}
assert(e.getCondition === "NOT_NULL_ASSERT_VIOLATION")
// Valid insert should succeed
sql("INSERT INTO t SELECT map('a', 1, 'b', 2)")
checkAnswer(spark.table("t"), Seq(Row(Map("a" -> 1, "b" -> 2))))
}
}
}

test("SPARK-55716: V1 DataFrame write ignores NOT NULL schema constraint") {
Seq("parquet", "orc").foreach { format =>
withTempPath { path =>
val data = Seq(Row(null, "hello", 1.0), Row(1, null, 2.0), Row(2, "world", null))
val df = spark.createDataFrame(
spark.sparkContext.parallelize(data),
new StructType()
.add("id", IntegerType, nullable = true)
.add("name", StringType, nullable = true)
.add("value", DoubleType, nullable = true))
// V1 DataSource writes do not enforce NOT NULL constraints
df.write.mode(SaveMode.Overwrite).format(format).save(path.getCanonicalPath)
val result = spark.read.format(format).load(path.getCanonicalPath)
checkAnswer(result, data)
}
}
}

test("UNSUPPORTED_OVERWRITE.PATH: Can't overwrite a path that is also being read from") {
val tableName = "t1"
withTable(tableName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -340,10 +340,12 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase {
case _ => assert(radix === 0) // nulls
}

assert(rowSet.getInt("NULLABLE") === 1)
val expectedNullable = if (schema(pos).nullable) 1 else 0
assert(rowSet.getInt("NULLABLE") === expectedNullable)
assert(rowSet.getString("REMARKS") === pos.toString)
assert(rowSet.getInt("ORDINAL_POSITION") === pos + 1)
assert(rowSet.getString("IS_NULLABLE") === "YES")
val expectedIsNullable = if (schema(pos).nullable) "YES" else "NO"
assert(rowSet.getString("IS_NULLABLE") === expectedIsNullable)
assert(rowSet.getString("IS_AUTO_INCREMENT") === "NO")
pos += 1
}
Expand Down