Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK] Support CTAS from a table with a deleted default value #4142

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
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
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import org.apache.spark.sql.delta.coordinatedcommits.CoordinatedCommitsUtils
import org.apache.spark.sql.delta.hooks.{HudiConverterHook, IcebergConverterHook, UpdateCatalog, UpdateCatalogFactory}
import org.apache.spark.sql.delta.logging.DeltaLogKeys
import org.apache.spark.sql.delta.metering.DeltaLogging
import org.apache.spark.sql.delta.schema.SchemaUtils
import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
Expand All @@ -40,12 +40,13 @@ import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.util.ResolveDefaultColumnsUtils
import org.apache.spark.sql.connector.catalog.Identifier
import org.apache.spark.sql.execution.command.{LeafRunnableCommand, RunnableCommand}
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.metric.SQLMetrics.createMetric
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType}

/**
* Single entry point for all write or declaration operations for Delta tables accessed through
Expand Down Expand Up @@ -314,7 +315,29 @@ case class CreateDeltaTableCommand(
txn.snapshot.domainMetadata, newDomainMetadata)
taggedCommitData = taggedCommitData.copy(actions = newActions)
}
val op = getOperation(txn.metadata, isManagedTable, Some(options),

def schemaContainsExistsColumnMetadata(schema: StructType): Boolean = {
schema.exists(
_.metadata.contains(ResolveDefaultColumnsUtils.EXISTS_DEFAULT_COLUMN_METADATA_KEY))
}
val metadataForOp =
Copy link
Contributor

Choose a reason for hiding this comment

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

This logic could use a comment to explain why we're doing it this way. IIUC we need to preserve the metadata on the source because the defaults are applied on read retroactively in the "exists default" case? So we keep the metadata on the source and we remove it from the target table, which makes it ineligible for DeltaTableUtils.removeInternalWriterMetadata and ColumnWithDefaultExprUtils.removeDefaultExpressions.

if (schemaContainsExistsColumnMetadata(txn.metadata.schema)) {
val newSchema = SchemaMergingUtils.transformColumns(schema) {
case (_, field: StructField, _) if field.metadata.contains(
Copy link
Contributor

Choose a reason for hiding this comment

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

Comment that default is only available as a top level field?

ResolveDefaultColumnsUtils.EXISTS_DEFAULT_COLUMN_METADATA_KEY) =>
val newMetadata = new MetadataBuilder()
.withMetadata(field.metadata)
.remove(ResolveDefaultColumnsUtils.EXISTS_DEFAULT_COLUMN_METADATA_KEY)
.build()
field.copy(metadata = newMetadata)
case (_, field: StructField, _) => field
}
txn.metadata.copy(schemaString = newSchema.json)
} else {
txn.metadata
}

val op = getOperation(metadataForOp, isManagedTable, Some(options),
clusterBy = ClusteredTableUtils.getLogicalClusteringColumnNames(
txn, taggedCommitData.actions)
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2409,6 +2409,31 @@ class DeltaTableCreationSuite
}
}
}

test("CREATE TABLE selecting from a table with removed column defaults") {
// Defaults are only possible for top level columns.
sql("""CREATE TABLE test_table(int_col INT DEFAULT 2)
|USING delta
|TBLPROPERTIES ('delta.feature.allowColumnDefaults' = 'supported')""".stripMargin)

def defaultsTableFeatureEnabled(tableName: String): Boolean = {
val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tableName))
val protocol = deltaLog.unsafeVolatileSnapshot.protocol
Comment on lines +2420 to +2421
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's avoid the unsafeVolatileSnapshot use

Suggested change
val deltaLog = DeltaLog.forTable(spark, TableIdentifier(tableName))
val protocol = deltaLog.unsafeVolatileSnapshot.protocol
val (_, snapshot) = DeltaLog.forTableWithSnapshot(spark, TableIdentifier(tableName))
val protocol = snapshot.protocol

protocol.writerFeatureNames.contains(AllowColumnDefaultsTableFeature.name)
}

assert(defaultsTableFeatureEnabled("test_table"))

Copy link
Contributor

Choose a reason for hiding this comment

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

Hm, so this EXISTS_DEFAULT_COLUMN_METADATA_KEY is applied retroactively on data IIUC. So it's not stored in the parquet files.

The semantics of CTAS are such that we want "a target table produced from reading the data of the source" which may not necessarily have the same target schema. I think we should have some data in this table to test that by removing the metadata where we are we still end up with a consistent target table.

We should also test when the column is added after data already exist via alter table, since it seems to go through a different logic. We should assert on the metadata we expect in the schema (CURRENT_DEFAULT_COLUMN_METADATA_KEY, EXISTS_DEFAULT_COLUMN_METADATA_KEY).

// @TODO: It is currently not possible to CTAS from a table with an active column default
// without explicitly enabling the table feature.
assertThrows[DeltaAnalysisException] {
sql("CREATE TABLE test_table_2 USING DELTA AS SELECT * FROM test_table")
}
sql("ALTER TABLE test_table ALTER COLUMN int_col DROP DEFAULT")
sql("CREATE TABLE test_table_3 USING DELTA AS SELECT * FROM test_table")

assert(!defaultsTableFeatureEnabled("test_table_3"))
}
}

trait DeltaTableCreationColumnMappingSuiteBase extends DeltaColumnMappingSelectedTestMixin {
Expand Down
Loading