Skip to content

Commit

Permalink
[UniForm] Cast Iceberg TIME to Spark Long (delta-io#4093)
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
This PR introduces the feature that allows UniForm to cast Iceberg TIME
type to Spark Long data type. The result will be represented as the
microseconds since midnight.

<!--
- 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?
UT
<!--
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?
No
<!--
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'.
-->
  • Loading branch information
harperjiang authored Jan 28, 2025
1 parent 0b7eed9 commit 2ffdbbf
Show file tree
Hide file tree
Showing 10 changed files with 330 additions and 23 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import org.apache.spark.sql.delta.DeltaColumnMapping
import org.apache.spark.sql.delta.schema.SchemaMergingUtils
import org.apache.iceberg.Schema
import org.apache.iceberg.spark.SparkSchemaUtil
import org.apache.iceberg.types.TypeUtil

import org.apache.spark.sql.types.{MetadataBuilder, StructType}

Expand All @@ -29,12 +30,18 @@ object IcebergSchemaUtils {
* Given an iceberg schema, convert it to a Spark schema. This conversion will keep the Iceberg
* column IDs (used to read Parquet files) in the field metadata
*
* @param icebergSchema
* @return StructType for the converted schema
* @param icebergSchema Iceberg schema
* @param castTimeType cast Iceberg TIME type to Spark Long
* @return Spark schema converted from Iceberg schema
*/
def convertIcebergSchemaToSpark(icebergSchema: Schema): StructType = {
def convertIcebergSchemaToSpark(icebergSchema: Schema,
castTimeType: Boolean = false): StructType = {
// Convert from Iceberg schema to Spark schema but without the column IDs
val baseConvertedSchema = SparkSchemaUtil.convert(icebergSchema)
val baseConvertedSchema = if (castTimeType) {
TypeUtil.visit(icebergSchema, new TypeToSparkTypeWithCustomCast()).asInstanceOf[StructType]
} else {
SparkSchemaUtil.convert(icebergSchema)
}

// For each field, find the column ID (fieldId) and add to the StructField metadata
SchemaMergingUtils.transformColumns(baseConvertedSchema) { (path, field, _) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import java.util.Locale

import scala.collection.JavaConverters._

import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaColumnMappingMode, DeltaConfigs, IdMapping, SerializableFileStatus}
import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaColumnMappingMode, DeltaConfigs, IdMapping, SerializableFileStatus, Snapshot}
import org.apache.spark.sql.delta.schema.SchemaMergingUtils
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.iceberg.{Table, TableProperties}
Expand All @@ -36,7 +36,7 @@ import org.apache.spark.sql.types.StructType
* A target Iceberg table for conversion to a Delta table.
*
* @param icebergTable the Iceberg table underneath.
* @param existingSchema schema used for incremental update, none for initial conversion.
* @param deltaSnapshot the delta snapshot used for incremental update, none for initial conversion.
* @param convertStats flag for disabling convert iceberg stats directly into Delta stats.
* If you wonder why we need this flag, you are not alone.
* This flag is only used by the old, obsolete, legacy command
Expand All @@ -49,23 +49,31 @@ import org.apache.spark.sql.types.StructType
class IcebergTable(
spark: SparkSession,
icebergTable: Table,
existingSchema: Option[StructType],
deltaSnapshot: Option[Snapshot],
convertStats: Boolean) extends ConvertTargetTable {

def this(spark: SparkSession, basePath: String, existingSchema: Option[StructType],
def this(spark: SparkSession, basePath: String, deltaTable: Option[Snapshot],
convertStats: Boolean = true) =
// scalastyle:off deltahadoopconfiguration
this(spark, new HadoopTables(spark.sessionState.newHadoopConf).load(basePath),
existingSchema, convertStats)
deltaTable, convertStats)
// scalastyle:on deltahadoopconfiguration

protected val existingSchema: Option[StructType] = deltaSnapshot.map(_.schema)

private val partitionEvolutionEnabled =
spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_PARTITION_EVOLUTION_ENABLED)

private val bucketPartitionEnabled =
spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_BUCKET_PARTITION_ENABLED)

private val fieldPathToPhysicalName =
// When a table is CLONED/federated with the session conf ON, it will have the table property
// set and will continue to support CAST TIME TYPE even when later the session conf is OFF.
private val castTimeType =
spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_CAST_TIME_TYPE) ||
deltaSnapshot.exists(s => DeltaConfigs.CAST_ICEBERG_TIME_TYPE.fromMetaData(s.metadata))

protected val fieldPathToPhysicalName =
existingSchema.map {
SchemaMergingUtils.explode(_).collect {
case (path, field) if DeltaColumnMapping.hasPhysicalName(field) =>
Expand All @@ -76,7 +84,7 @@ class IcebergTable(
private val convertedSchema = {
// Reuse physical names of existing columns.
val mergedSchema = DeltaColumnMapping.setPhysicalNames(
IcebergSchemaUtils.convertIcebergSchemaToSpark(icebergTable.schema()),
IcebergSchemaUtils.convertIcebergSchemaToSpark(icebergTable.schema(), castTimeType),
fieldPathToPhysicalName)

// Assign physical names to new columns.
Expand All @@ -88,8 +96,13 @@ class IcebergTable(
override val properties: Map[String, String] = {
val maxSnapshotAgeMs = PropertyUtil.propertyAsLong(icebergTable.properties,
TableProperties.MAX_SNAPSHOT_AGE_MS, TableProperties.MAX_SNAPSHOT_AGE_MS_DEFAULT)
val castTimeTypeConf = if (castTimeType) {
Some((DeltaConfigs.CAST_ICEBERG_TIME_TYPE.key -> "true"))
} else {
None
}
icebergTable.properties().asScala.toMap + (DeltaConfigs.COLUMN_MAPPING_MODE.key -> "id") +
(DeltaConfigs.LOG_RETENTION.key -> s"$maxSnapshotAgeMs millisecond")
(DeltaConfigs.LOG_RETENTION.key -> s"$maxSnapshotAgeMs millisecond") ++ castTimeTypeConf
}

override val partitionSchema: StructType = {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
/*
* 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.commands.convert

import java.util

import scala.collection.JavaConverters._

import org.apache.iceberg.MetadataColumns
import org.apache.iceberg.Schema
import org.apache.iceberg.relocated.com.google.common.collect.Lists
import org.apache.iceberg.types.Type
import org.apache.iceberg.types.Type.TypeID._
import org.apache.iceberg.types.Types
import org.apache.iceberg.types.TypeUtil

import org.apache.spark.sql.types.ArrayType
import org.apache.spark.sql.types.BinaryType
import org.apache.spark.sql.types.BooleanType
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.types.DateType
import org.apache.spark.sql.types.DecimalType
import org.apache.spark.sql.types.DoubleType
import org.apache.spark.sql.types.FloatType
import org.apache.spark.sql.types.IntegerType
import org.apache.spark.sql.types.LongType
import org.apache.spark.sql.types.MapType
import org.apache.spark.sql.types.Metadata
import org.apache.spark.sql.types.MetadataBuilder
import org.apache.spark.sql.types.StringType
import org.apache.spark.sql.types.StructField
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.types.TimestampNTZType
import org.apache.spark.sql.types.TimestampType

/**
* This class is copied from [[org.apache.iceberg.spark.TypeToSparkType]] to
* add custom type casting. Currently, it supports the following casting
* * Iceberg TIME -> Spark Long
*
*/
class TypeToSparkTypeWithCustomCast extends TypeUtil.SchemaVisitor[DataType] {

val METADATA_COL_ATTR_KEY = "__metadata_col";

override def schema(schema: Schema, structType: DataType): DataType = structType

override def struct(struct: Types.StructType, fieldResults: util.List[DataType]): DataType = {
val fields = struct.fields();
val sparkFields: util.List[StructField] =
Lists.newArrayListWithExpectedSize(fieldResults.size())
for (i <- 0 until fields.size()) {
val field = fields.get(i)
val `type` = fieldResults.get(i)
val metadata = fieldMetadata(field.fieldId())
var sparkField = StructField.apply(field.name(), `type`, field.isOptional(), metadata)
if (field.doc() != null) {
sparkField = sparkField.withComment(field.doc())
}
sparkFields.add(sparkField)
}

StructType.apply(sparkFields)
}

override def field(field: Types.NestedField, fieldResult: DataType): DataType = fieldResult

override def list(list: Types.ListType, elementResult: DataType): DataType =
ArrayType.apply(elementResult, list.isElementOptional())

override def map(map: Types.MapType, keyResult: DataType, valueResult: DataType): DataType =
MapType.apply(keyResult, valueResult, map.isValueOptional())

override def primitive(primitive: Type.PrimitiveType): DataType = {
primitive.typeId() match {
case BOOLEAN => BooleanType
case INTEGER => IntegerType
case LONG => LongType
case FLOAT => FloatType
case DOUBLE => DoubleType
case DATE => DateType
// This line is changed to allow casting TIME to Spark Long.
// The result is microseconds since midnight.
case TIME => LongType
case TIMESTAMP =>
val ts = primitive.asInstanceOf[Types.TimestampType]
if (ts.shouldAdjustToUTC()) {
TimestampType
} else {
TimestampNTZType
}
case STRING => StringType
case UUID => // use String
StringType
case FIXED => BinaryType
case BINARY => BinaryType
case DECIMAL =>
val decimal = primitive.asInstanceOf[Types.DecimalType]
DecimalType.apply(decimal.precision(), decimal.scale());
case _ =>
throw new UnsupportedOperationException(
"Cannot convert unknown type to Spark: " + primitive);
}
}

private def fieldMetadata(fieldId: Int): Metadata = {
if (MetadataColumns.metadataFieldIds().contains(fieldId)) {
return new MetadataBuilder().putBoolean(METADATA_COL_ATTR_KEY, value = true).build()
}

Metadata.empty
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ package org.apache.spark.sql.delta

// scalastyle:off import.ordering.noEmptyLine
import java.sql.Date
import java.time.LocalTime

import scala.collection.JavaConverters._
import scala.util.Try
Expand All @@ -26,7 +27,10 @@ import org.apache.spark.sql.delta.commands.convert.ConvertUtils
import org.apache.spark.sql.delta.schema.SchemaMergingUtils
import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.spark.sql.delta.stats.StatisticsCollection
import org.apache.iceberg.Schema
import org.apache.iceberg.hadoop.HadoopTables
import org.apache.iceberg.types.Types
import org.apache.iceberg.types.Types.NestedField

import org.apache.spark.SparkConf
import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row}
Expand Down Expand Up @@ -451,6 +455,58 @@ class CloneIcebergByPathSuite extends CloneIcebergSuiteBase
}
}

/**
* This suite test features in Iceberg that is not directly supported by Spark.
* See also [[NonSparkIcebergTestUtils]].
* We do not put these tests in or extend from [[CloneIcebergSuiteBase]] because they
* use non-Spark way to create test data.
*/
class CloneNonSparkIcebergByPathSuite extends QueryTest
with ConvertIcebergToDeltaUtils {

protected val cloneTable = "clone"

private def sourceIdentifier: String = s"iceberg.`$tablePath`"

private def runCreateOrReplace(mode: String, source: String): DataFrame = {
Try(spark.sql(s"DELETE FROM $cloneTable"))
spark.sql(s"CREATE OR REPLACE TABLE $cloneTable $mode CLONE $source")
}

private val mode = "SHALLOW"

test("cast Iceberg TIME to Spark long") {
withTable(table, cloneTable) {
val schema = new Schema(
Seq[NestedField](
NestedField.required(1, "id", Types.IntegerType.get),
NestedField.required(2, "event_time", Types.TimeType.get)
).asJava
)
val rows = Seq(
Map(
"id" -> 1,
"event_time" -> LocalTime.of(14, 30, 11)
)
)
NonSparkIcebergTestUtils.createIcebergTable(spark, tablePath, schema, rows)
intercept[UnsupportedOperationException] {
runCreateOrReplace(mode, sourceIdentifier)
}
withSQLConf(DeltaSQLConf.DELTA_CONVERT_ICEBERG_CAST_TIME_TYPE.key -> "true") {
runCreateOrReplace(mode, sourceIdentifier)
val expectedMicrosec = (14 * 3600 + 30 * 60 + 11) * 1000000L
checkAnswer(spark.table(cloneTable), Row(1, expectedMicrosec) :: Nil)
val clonedDeltaTable = DeltaLog.forTable(
spark,
spark.sessionState.catalog.getTableMetadata(TableIdentifier(cloneTable))
)
assert(DeltaConfigs.CAST_ICEBERG_TIME_TYPE.fromMetaData(clonedDeltaTable.update().metadata))
}
}
}
}

class CloneIcebergByNameSuite extends CloneIcebergSuiteBase
{
override def sourceIdentifier: String = table
Expand Down
Loading

0 comments on commit 2ffdbbf

Please sign in to comment.