@@ -19,9 +19,12 @@ package org.apache.spark.sql.delta
1919// scalastyle:off import.ordering.noEmptyLine
2020import java .io .PrintWriter
2121
22+ import scala .collection .JavaConverters ._
23+
2224import org .apache .spark .sql .delta .commands .cdc .CDCReader
23- import org .apache .spark .sql .delta .schema .{InvariantViolationException , SchemaUtils }
25+ import org .apache .spark .sql .delta .schema .{DeltaInvariantViolationException , InvariantViolationException , SchemaUtils }
2426import org .apache .spark .sql .delta .sources .DeltaSourceUtils .GENERATION_EXPRESSION_METADATA_KEY
27+ import org .apache .spark .sql .delta .sources .DeltaSQLConf
2528import org .apache .spark .sql .delta .test .DeltaSQLCommandTest
2629import io .delta .tables .DeltaTableBuilder
2730
@@ -35,7 +38,7 @@ import org.apache.spark.sql.functions.{current_timestamp, lit}
3538import org .apache .spark .sql .internal .SQLConf
3639import org .apache .spark .sql .streaming .{StreamingQueryException , Trigger }
3740import org .apache .spark .sql .test .SharedSparkSession
38- import org .apache .spark .sql .types .{ArrayType , DateType , IntegerType , MetadataBuilder , StructField , StructType , TimestampType }
41+ import org .apache .spark .sql .types .{ArrayType , DateType , IntegerType , MetadataBuilder , StringType , StructField , StructType , TimestampType }
3942import org .apache .spark .unsafe .types .UTF8String
4043
4144trait GeneratedColumnTest extends QueryTest with SharedSparkSession with DeltaSQLCommandTest {
@@ -1672,6 +1675,67 @@ trait GeneratedColumnSuiteBase extends GeneratedColumnTest {
16721675 )
16731676 }
16741677 }
1678+
1679+ test(" not null should be enforced with generated columns" ) {
1680+ withTableName(" tbl" ) { tbl =>
1681+ createTable(tbl,
1682+ None , " c1 INT, c2 STRING, c3 INT" , Map (" c3" -> " c1 + 1" ), Seq .empty, Set (" c1" , " c2" , " c3" ))
1683+
1684+ // try to write data without c2 in the DF
1685+ val schemaWithoutColumnC2 = StructType (
1686+ Seq (StructField (" c1" , IntegerType , true )))
1687+ val data1 = List (Row (3 ))
1688+ val df1 = spark.createDataFrame(data1.asJava, schemaWithoutColumnC2)
1689+
1690+ val e1 = intercept[DeltaInvariantViolationException ] {
1691+ df1.write.format(" delta" ).mode(" append" ).saveAsTable(" tbl" )
1692+ }
1693+ assert(e1.getMessage.contains(" Column c2, which has a NOT NULL constraint," +
1694+ " is missing from the data being written into the table." ))
1695+ }
1696+ }
1697+
1698+ Seq (true , false ).foreach { allowNullInsert =>
1699+ test(" nullable column should work with generated columns - " +
1700+ " allowNullInsert enabled=" + allowNullInsert) {
1701+ withTableName(" tbl" ) { tbl =>
1702+ withSQLConf(DeltaSQLConf .GENERATED_COLUMN_ALLOW_NULLABLE .key -> allowNullInsert.toString) {
1703+ createTable(
1704+ tbl, None , " c1 INT, c2 STRING, c3 INT" , Map (" c3" -> " c1 + 1" ), Seq .empty)
1705+
1706+ // create data frame that matches the table's schema
1707+ val data1 = List (Row (1 , " a1" ), Row (2 , " a2" ))
1708+ val schema = StructType (
1709+ Seq (StructField (" c1" , IntegerType , true ), StructField (" c2" , StringType , true )))
1710+ val df1 = spark.createDataFrame(data1.asJava, schema)
1711+ df1.write.format(" delta" ).mode(" append" ).saveAsTable(" tbl" )
1712+
1713+ // create a data frame that does not have c2
1714+ val schemaWithoutOptionalColumnC2 = StructType (
1715+ Seq (StructField (" c1" , IntegerType , true )))
1716+
1717+ val data2 = List (Row (3 ))
1718+ val df2 = spark.createDataFrame(data2.asJava, schemaWithoutOptionalColumnC2)
1719+
1720+ if (allowNullInsert) {
1721+ df2.write.format(" delta" ).mode(" append" ).saveAsTable(" tbl" )
1722+ // check correctness
1723+ val expectedDF = df1
1724+ .union(df2.withColumn(" c2" , lit(null ).cast(StringType )))
1725+ .withColumn(" c3" , ' c1 + 1 )
1726+ checkAnswer(spark.read.table(tbl), expectedDF)
1727+ } else {
1728+ // when allow null insert is not enabled.
1729+ val e = intercept[AnalysisException ] {
1730+ df2.write.format(" delta" ).mode(" append" ).saveAsTable(" tbl" )
1731+ }
1732+ e.getMessage.contains(
1733+ " A column or function parameter with name `c2` cannot be resolved" )
1734+ }
1735+ }
1736+ }
1737+ }
1738+ }
16751739}
16761740
16771741class GeneratedColumnSuite extends GeneratedColumnSuiteBase
0 commit comments