Skip to content
Closed
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/docs/spark-configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,7 @@ val spark = SparkSession.builder()
| spark.sql.iceberg.merge-schema | false | Enables modifying the table schema to match the write schema. Only adds columns missing columns |
| spark.sql.iceberg.report-column-stats | true | Report Puffin Table Statistics if available to Spark's Cost Based Optimizer. CBO must be enabled for this to be effective |
| spark.sql.iceberg.async-micro-batch-planning-enabled | false | Enables asynchronous microbatch planning to reduce planning latency by pre-fetching file scan tasks |
| spark.sql.iceberg.read.as-of-timestamp | null | Default timestamp in milliseconds for time-travel queries; applies to all reads when no explicit snapshot-id, as-of-timestamp, branch, or tag is specified |

### Read options

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ package org.apache.iceberg.spark.extensions
import org.apache.spark.sql.SparkSessionExtensions
import org.apache.spark.sql.catalyst.analysis.CheckViews
import org.apache.spark.sql.catalyst.analysis.ResolveBranch
import org.apache.spark.sql.catalyst.analysis.ResolveDefaultTimestamp
import org.apache.spark.sql.catalyst.analysis.ResolveViews
import org.apache.spark.sql.catalyst.optimizer.ReplaceStaticInvoke
import org.apache.spark.sql.catalyst.parser.extensions.IcebergSparkSqlExtensionsParser
Expand All @@ -35,6 +36,7 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
// analyzer extensions
extensions.injectResolutionRule { spark => ResolveViews(spark) }
extensions.injectPostHocResolutionRule { spark => ResolveBranch(spark) }
extensions.injectPostHocResolutionRule { spark => ResolveDefaultTimestamp(spark) }
extensions.injectCheckRule(_ => CheckViews)

// optimizer extensions
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.catalyst.analysis

import org.apache.iceberg.spark.PathIdentifier
import org.apache.iceberg.spark.SparkSQLProperties
import org.apache.iceberg.spark.TimeTravel
import org.apache.iceberg.spark.source.SparkTable
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.catalog.Identifier
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation

case class ResolveDefaultTimestamp(spark: SparkSession) extends Rule[LogicalPlan] {

override def apply(plan: LogicalPlan): LogicalPlan = {
val sessionTimestamp = spark.conf.getOption(SparkSQLProperties.AS_OF_TIMESTAMP)
if (sessionTimestamp.isEmpty) return plan

val timestampMillis = sessionTimestamp.get.toLong

plan resolveOperators {
// SQL-level time travel (TIMESTAMP AS OF / VERSION AS OF branch) conflicts with session property
case DataSourceV2Relation(table: SparkTable, _, _, _, _, Some(_)) if table.branch() != null =>
throw new IllegalArgumentException(
s"Cannot override ref, already set snapshot id=${table.snapshotId()}")

// SQL-level time travel (TIMESTAMP AS OF / VERSION AS OF snapshot or tag) conflicts
case DataSourceV2Relation(_: SparkTable, _, _, _, _, Some(_)) =>
throw new IllegalArgumentException(
"Cannot set both snapshot-id and as-of-timestamp to select which table snapshot to scan")

// Catalog-level branch selection (e.g. branch_X identifier set by ResolveBranch) conflicts
case DataSourceV2Relation(table: SparkTable, _, _, _, _, None) if table.branch() != null =>
throw new IllegalArgumentException(
s"Cannot override ref, already set snapshot id=${table.snapshotId()}")

// Catalog-level snapshot/tag/timestamp selector (e.g. snapshot_id_X, tag_X) conflicts
case DataSourceV2Relation(table: SparkTable, _, _, ident, _, None)
if table.branch() == null && hasExplicitSelector(ident) =>
throw new IllegalArgumentException(
"Cannot set both snapshot-id and as-of-timestamp to select which table snapshot to scan")

// Plain read: apply session timestamp and encode the resolved snapshot in the identifier
case r @ DataSourceV2Relation(table: SparkTable, _, _, ident, _, None)
if table.branch() == null =>
val newTable = SparkTable.create(table.table(), TimeTravel.timestampMillis(timestampMillis))
val snapshotSelector = s"snapshot_id_${newTable.snapshotId()}"
val newIdent = ident.map {
case path: PathIdentifier if path.location.contains("#") =>
new PathIdentifier(path.location + "," + snapshotSelector)
case path: PathIdentifier =>
new PathIdentifier(path.location + "#" + snapshotSelector)
case i =>
Identifier.of(i.namespace() :+ i.name(), snapshotSelector)
}
r.copy(table = newTable, identifier = newIdent)
}
}

private def hasExplicitSelector(ident: Option[Identifier]): Boolean =
ident.exists { i =>
val name = i.name()
name.startsWith("snapshot_id_") || name.startsWith("at_timestamp_") || name.startsWith("tag_")
}
}
Loading