Skip to content

Commit

Permalink
[Spark] Add read support for RowId (delta-io#2856)
Browse files Browse the repository at this point in the history
<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [x] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [ ] Other (fill in here)

## Description
1. Add the Analyzer Rule `GenerateRowIds` to generate default Row IDs.
2. Add the `row_id` field to the `_metadata` column for Delta tables,
allowing us to read the `row_id` from the file metadata after it is
stored.
<!--
- Describe what this PR changes.
- Describe why we need the change.
 
If this PR resolves an issue be sure to include "Resolves #XXX" to
correctly link and close the issue upon merge.
-->

## How was this patch tested?
Added UTs.
<!--
If tests were added, say they were added here. Please make sure to test
the changes thoroughly including negative and positive cases if
possible.
If the changes were tested in any way other than unit tests, please
clarify how you tested step by step (ideally copy and paste-able, so
that other reviewers can test and check, and descendants can verify in
the future).
If the changes were not tested, please explain why.
-->

## Does this PR introduce _any_ user-facing changes?

<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->
No.
  • Loading branch information
longvu-db authored and andreaschat-db committed Apr 16, 2024
1 parent b0ab2e6 commit b132344
Show file tree
Hide file tree
Showing 15 changed files with 915 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,9 @@ class DeltaSparkSessionExtension extends (SparkSessionExtensions => Unit) {
extensions.injectPostHocResolutionRule { session =>
PostHocResolveUpCast(session)
}

extensions.injectPlanNormalizationRule { _ => GenerateRowIDs }

// We don't use `injectOptimizerRule` here as we won't want to apply further optimizations after
// `PrepareDeltaScan`.
// For example, `ConstantFolding` will break unit tests in `OptimizeGeneratedColumnSuite`.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,12 @@ object ColumnWithDefaultExprUtils extends DeltaLogging {
}
}
selectExprs = selectExprs ++ cdcSelectExprs

val rowIdExprs = data.queryExecution.analyzed.output
.filter(RowId.RowIdMetadataAttribute.isRowIdColumn)
.map(new Column(_))
selectExprs = selectExprs ++ rowIdExprs

val newData = queryExecution match {
case incrementalExecution: IncrementalExecution =>
selectFromStreamingDataFrame(incrementalExecution, data, selectExprs: _*)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import java.util.{Locale, UUID}

import scala.collection.mutable

import org.apache.spark.sql.delta.RowId.RowIdMetadataStructField
import org.apache.spark.sql.delta.actions.{Metadata, Protocol}
import org.apache.spark.sql.delta.commands.cdc.CDCReader
import org.apache.spark.sql.delta.metering.DeltaLogging
Expand Down Expand Up @@ -78,8 +79,9 @@ trait DeltaColumnMappingBase extends DeltaLogging {
val supportedModes: Set[DeltaColumnMappingMode] =
Set(IdMapping, NoMapping, NameMapping)

def isInternalField(field: StructField): Boolean = DELTA_INTERNAL_COLUMNS
.contains(field.name.toLowerCase(Locale.ROOT))
def isInternalField(field: StructField): Boolean =
DELTA_INTERNAL_COLUMNS.contains(field.name.toLowerCase(Locale.ROOT)) ||
RowIdMetadataStructField.isRowIdColumn(field)

def satisfiesColumnMappingProtocol(protocol: Protocol): Boolean =
protocol.isFeatureSupported(ColumnMappingTableFeature)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ import org.apache.spark.util.SerializableConfiguration
case class DeltaParquetFileFormat(
protocol: Protocol,
metadata: Metadata,
nullableRowTrackingFields: Boolean = false,
isSplittable: Boolean = true,
disablePushDowns: Boolean = false,
tablePath: Option[String] = None,
Expand Down Expand Up @@ -197,12 +198,21 @@ case class DeltaParquetFileFormat(
}

override def metadataSchemaFields: Seq[StructField] = {
// Parquet reader in Spark has a bug where a file containing 2b+ rows in a single rowgroup
// causes it to run out of the `Integer` range (TODO: Create a SPARK issue)
val rowTrackingFields =
RowTracking.createMetadataStructFields(protocol, metadata, nullableRowTrackingFields)
// TODO(SPARK-47731): Parquet reader in Spark has a bug where a file containing 2b+ rows
// in a single rowgroup causes it to run out of the `Integer` range.
// For Delta Parquet readers don't expose the row_index field as a metadata field.
super.metadataSchemaFields.filter(field => field != ParquetFileFormat.ROW_INDEX_FIELD) ++
RowId.createBaseRowIdField(protocol, metadata) ++
DefaultRowCommitVersion.createDefaultRowCommitVersionField(protocol, metadata)
if (!RowId.isEnabled(protocol, metadata)) {
super.metadataSchemaFields.filter(_ != ParquetFileFormat.ROW_INDEX_FIELD)
} else {
// It is fine to expose the row_index field as a metadata field when Row Tracking
// is enabled because it is needed to generate the Row ID field, and it is not a
// big problem if we use 2b+ rows in a single rowgroup, it will throw an exception and
// we can then use less rows per rowgroup. Also, 2b+ rows in a single rowgroup is
// not a common use case.
super.metadataSchemaFields ++ rowTrackingFields
}
}

override def prepareWrite(
Expand Down
139 changes: 139 additions & 0 deletions spark/src/main/scala/org/apache/spark/sql/delta/GenerateRowIDs.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,139 @@
/*
* Copyright (2021) The Delta Lake Project Authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.delta

import scala.collection.mutable

import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation, LogicalRelation}
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.types.StructType

/**
* This rule adds a Project on top of Delta tables that support the Row tracking table feature to
* provide a default generated Row ID for rows that don't have them materialized in the data file.
*/
object GenerateRowIDs extends Rule[LogicalPlan] {

/**
* Matcher for a scan on a Delta table that has Row tracking enabled.
*/
private object DeltaScanWithRowTrackingEnabled {
def unapply(plan: LogicalPlan): Option[LogicalRelation] = plan match {
case scan @ LogicalRelation(relation: HadoopFsRelation, _, _, _) =>
relation.fileFormat match {
case format: DeltaParquetFileFormat
if RowTracking.isEnabled(format.protocol, format.metadata) => Some(scan)
case _ => None
}
case _ => None
}
}

override def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithNewOutput {
case DeltaScanWithRowTrackingEnabled(scan) =>
// While Row IDs are non-nullable, we'll use the Row ID attributes to read
// the materialized values from now on, which can be null. We make
// the materialized Row ID attributes nullable in the scan here.

// Update nullability in the scan `metadataOutput` by updating the delta file format.
val baseRelation = scan.relation.asInstanceOf[HadoopFsRelation]
val newFileFormat = baseRelation.fileFormat match {
case format: DeltaParquetFileFormat =>
format.copy(nullableRowTrackingFields = true)
}
val newBaseRelation = baseRelation.copy(fileFormat = newFileFormat)(baseRelation.sparkSession)

// Update the output metadata column's data type (now with nullable row tracking fields).
val newOutput = scan.output.map {
case MetadataAttributeWithLogicalName(metadata, FileFormat.METADATA_NAME) =>
metadata.withDataType(newFileFormat.createFileMetadataCol().dataType)
case other => other
}
val newScan = scan.copy(relation = newBaseRelation, output = newOutput)
newScan.copyTagsFrom(scan)

// Add projection with row tracking column expressions.
val updatedAttributes = mutable.Buffer.empty[(Attribute, Attribute)]
val projectList = newOutput.map {
case MetadataAttributeWithLogicalName(metadata, FileFormat.METADATA_NAME) =>
val updatedMetadata = metadataWithRowTrackingColumnsProjection(metadata)
updatedAttributes += metadata -> updatedMetadata.toAttribute
updatedMetadata
case other => other
}
Project(projectList = projectList, child = newScan) -> updatedAttributes.toSeq
case o =>
val newPlan = o.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) {
// Recurse into subquery plans. Similar to how [[transformUpWithSubqueries]] works except
// that it allows us to still use [[transformUpWithNewOutput]] on subquery plans to
// correctly update references to the metadata attribute when going up the plan.
// Get around type erasure by explicitly checking the plan type and removing warning.
case planExpression: PlanExpression[LogicalPlan @unchecked]
if planExpression.plan.isInstanceOf[LogicalPlan] =>
planExpression.withNewPlan(apply(planExpression.plan))
}
newPlan -> Nil
}

/**
* Expression that reads the Row IDs from the materialized Row ID column if the value is
* present and returns the default generated Row ID using the file's base Row ID and current row
* index if not:
* coalesce(_metadata.row_id, _metadata.base_row_id + _metadata.row_index).
*/
private def rowIdExpr(metadata: AttributeReference): Expression = {
Coalesce(Seq(
getField(metadata, RowId.ROW_ID),
Add(
getField(metadata, RowId.BASE_ROW_ID),
getField(metadata, ParquetFileFormat.ROW_INDEX))))
}

/**
* Extract a field from the metadata column.
*/
private def getField(metadata: AttributeReference, name: String): GetStructField = {
ExtractValue(metadata, Literal(name), conf.resolver) match {
case field: GetStructField => field
case _ =>
throw new IllegalStateException(s"The metadata column '${metadata.name}' is not a struct.")
}
}

/**
* Create a new metadata struct where the Row ID values are populated using
* the materialized values if present, or the default Row ID values if not.
*/
private def metadataWithRowTrackingColumnsProjection(metadata: AttributeReference)
: NamedExpression = {
val metadataFields = metadata.dataType.asInstanceOf[StructType].map {
case field if field.name == RowId.ROW_ID =>
field -> rowIdExpr(metadata)
case field =>
field -> getField(metadata, field.name)
}.flatMap { case (oldField, newExpr) =>
// Propagate the type metadata from the old fields to the new fields.
val newField = Alias(newExpr, oldField.name)(explicitMetadata = Some(oldField.metadata))
Seq(Literal(oldField.name), newField)
}
Alias(CreateNamedStruct(metadataFields), metadata.name)()
}
}
115 changes: 114 additions & 1 deletion spark/src/main/scala/org/apache/spark/sql/delta/RowId.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,11 @@ import org.apache.spark.sql.delta.actions.{Action, AddFile, DomainMetadata, Meta
import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils.propertyKey
import org.apache.spark.sql.util.ScalaExtensions._

import org.apache.spark.sql.catalyst.expressions.FileSourceConstantMetadataStructField
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, FileSourceConstantMetadataStructField, FileSourceGeneratedMetadataStructField}
import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
import org.apache.spark.sql.execution.datasources.FileFormat
import org.apache.spark.sql.types
import org.apache.spark.sql.types.{DataType, LongType, MetadataBuilder, StructField}

Expand Down Expand Up @@ -155,4 +159,113 @@ object RowId {
Option.when(RowId.isEnabled(protocol, metadata)) {
BaseRowIdMetadataStructField()
}

/** Row ID column name */
val ROW_ID = "row_id"

val QUALIFIED_COLUMN_NAME = s"${FileFormat.METADATA_NAME}.${ROW_ID}"

/** Column metadata to be used in conjunction [[QUALIFIED_COLUMN_NAME]] to mark row id columns */
def columnMetadata(materializedColumnName: String): types.Metadata =
RowIdMetadataStructField.metadata(materializedColumnName)

/**
* The field readers can use to access the generated row id column. The scanner's internal column
* name is obtained from the table's metadata.
*/
def createRowIdField(protocol: Protocol, metadata: Metadata, nullable: Boolean)
: Option[StructField] =
MaterializedRowId.getMaterializedColumnName(protocol, metadata)
.map(RowIdMetadataStructField(_, nullable))

/*
* A specialization of [[FileSourceGeneratedMetadataStructField]] used to represent RowId columns.
*
* - Row ID columns can be read by adding '_metadata.row_id' to the read schema
* - To write to the materialized Row ID column
* - use the materialized Row ID column name which can be obtained using
* [[getMaterializedColumnName]]
* - add [[COLUMN_METADATA]] which is part of [[RowId]] as metadata to the column
* - nulls are replaced with fresh Row IDs
*/
object RowIdMetadataStructField {

val ROW_ID_METADATA_COL_ATTR_KEY = "__row_id_metadata_col"

def metadata(materializedColumnName: String): types.Metadata = new MetadataBuilder()
.withMetadata(
FileSourceGeneratedMetadataStructField.metadata(RowId.ROW_ID, materializedColumnName))
.putBoolean(ROW_ID_METADATA_COL_ATTR_KEY, value = true)
.build()

def apply(materializedColumnName: String, nullable: Boolean = false): StructField =
StructField(
RowId.ROW_ID,
LongType,
// The Row ID field is used to read the materialized Row ID value which is nullable. The
// actual Row ID expression is created using a projection injected before the optimizer pass
// by the [[GenerateRowIDs] rule at which point the Row ID field is non-nullable.
nullable,
metadata = metadata(materializedColumnName))

def unapply(field: StructField): Option[StructField] =
if (isRowIdColumn(field)) Some(field) else None

/** Return true if the column is a Row Id column. */
def isRowIdColumn(structField: StructField): Boolean =
isValid(structField.dataType, structField.metadata)

def isValid(dataType: DataType, metadata: types.Metadata): Boolean = {
FileSourceGeneratedMetadataStructField.isValid(dataType, metadata) &&
metadata.contains(ROW_ID_METADATA_COL_ATTR_KEY) &&
metadata.getBoolean(ROW_ID_METADATA_COL_ATTR_KEY)
}
}

object RowIdMetadataAttribute {
/** Creates an attribute for writing out the materialized column name */
def apply(materializedColumnName: String): AttributeReference =
DataTypeUtils.toAttribute(RowIdMetadataStructField(materializedColumnName))
.withName(materializedColumnName)

def unapply(attr: Attribute): Option[Attribute] =
if (isRowIdColumn(attr)) Some(attr) else None

/** Return true if the column is a Row Id column. */
def isRowIdColumn(attr: Attribute): Boolean =
RowIdMetadataStructField.isValid(attr.dataType, attr.metadata)
}

/**
* Throw if row tracking is supported and columns in the write schema tagged as materialized row
* IDs do not reference the materialized row id column name.
*/
private[delta] def throwIfMaterializedRowIdColumnNameIsInvalid(
data: DataFrame, metadata: Metadata, protocol: Protocol, tableId: String): Unit = {
if (!RowTracking.isEnabled(protocol, metadata)) {
return
}

val materializedColumnName =
metadata.configuration.get(MaterializedRowId.MATERIALIZED_COLUMN_NAME_PROP)

if (materializedColumnName.isEmpty) {
// If row tracking is enabled, a missing materialized column name is a bug and we need to
// throw an error. If row tracking is only supported, we should just return, as it's fine
// for the materialized column to not be assigned.
if (RowTracking.isEnabled(protocol, metadata)) {
throw DeltaErrors.materializedRowIdMetadataMissing(tableId)
}
return
}

toAttributes(data.schema).foreach {
case RowIdMetadataAttribute(attribute) =>
if (attribute.name != materializedColumnName.get) {
throw new UnsupportedOperationException("Materialized Row IDs column name " +
s"${attribute.name} is invalid. Must be ${materializedColumnName.get}.")
}
case _ =>
}
}
}
12 changes: 12 additions & 0 deletions spark/src/main/scala/org/apache/spark/sql/delta/RowTracking.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ package org.apache.spark.sql.delta

import org.apache.spark.sql.delta.actions.{Metadata, Protocol, TableFeatureProtocolUtils}

import org.apache.spark.sql.types.StructField

/**
* Utility functions for Row Tracking that are shared between Row IDs and Row Commit Versions.
Expand Down Expand Up @@ -60,4 +61,15 @@ object RowTracking {
throw DeltaErrors.convertToDeltaRowTrackingEnabledWithoutStatsCollection
}
}

/**
* Returns the Row Tracking metadata fields for the file's _metadata when Row Tracking
* is enabled.
*/
def createMetadataStructFields(protocol: Protocol, metadata: Metadata, nullable: Boolean)
: Iterable[StructField] = {
RowId.createRowIdField(protocol, metadata, nullable) ++
RowId.createBaseRowIdField(protocol, metadata) ++
DefaultRowCommitVersion.createDefaultRowCommitVersionField(protocol, metadata)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,11 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl
val normalizedData = SchemaUtils.normalizeColumnNames(
deltaLog, metadata.schema, data
)

// Validate that write columns for Row IDs have the correct name.
RowId.throwIfMaterializedRowIdColumnNameIsInvalid(
normalizedData, metadata, protocol, deltaLog.tableId)

val nullAsDefault = options.isDefined &&
options.get.options.contains(ColumnWithDefaultExprUtils.USE_NULL_AS_DEFAULT_DELTA_OPTION)
val enforcesDefaultExprs = ColumnWithDefaultExprUtils.tableHasDefaultExpr(
Expand Down
Loading

0 comments on commit b132344

Please sign in to comment.