Skip to content

Commit ef1def9

Browse files
authored
[Spark] Support RESTORE for clustered table (#3194)
## Description Support RESTORE for clustered tables by adding a new domain metadata to overwrite the existing one so that clustering columns are correctly restored. ## How was this patch tested? New unit tests.
1 parent 87549c5 commit ef1def9

File tree

5 files changed

+151
-19
lines changed

5 files changed

+151
-19
lines changed

spark/src/main/scala/org/apache/spark/sql/delta/DomainMetadataUtils.scala

Lines changed: 47 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -16,20 +16,26 @@
1616

1717
package org.apache.spark.sql.delta
1818

19+
import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils
1920
import org.apache.spark.sql.delta.actions.{Action, DomainMetadata, Protocol}
2021
import org.apache.spark.sql.delta.clustering.ClusteringMetadataDomain
2122
import org.apache.spark.sql.delta.metering.DeltaLogging
2223

23-
object DomainMetadataUtils extends DeltaLogging {
24+
/**
25+
* Domain metadata utility functions.
26+
*/
27+
trait DomainMetadataUtilsBase extends DeltaLogging {
2428
// List of metadata domains that will be removed for the REPLACE TABLE operation.
25-
private val METADATA_DOMAINS_TO_REMOVE_FOR_REPLACE_TABLE: Set[String] = Set(
26-
)
29+
protected val METADATA_DOMAINS_TO_REMOVE_FOR_REPLACE_TABLE: Set[String] = Set(
30+
ClusteringMetadataDomain.domainName)
31+
2732
// List of metadata domains that will be copied from the table we are restoring to.
28-
private val METADATA_DOMAIN_TO_COPY_FOR_RESTORE_TABLE =
29-
METADATA_DOMAINS_TO_REMOVE_FOR_REPLACE_TABLE
33+
// Note that ClusteringMetadataDomain are recreated in handleDomainMetadataForRestoreTable
34+
// instead of being blindly copied over.
35+
protected val METADATA_DOMAIN_TO_COPY_FOR_RESTORE_TABLE: Set[String] = Set.empty
3036

3137
// List of metadata domains that will be copied from the table on a CLONE operation.
32-
private val METADATA_DOMAIN_TO_COPY_FOR_CLONE_TABLE: Set[String] = Set(
38+
protected val METADATA_DOMAIN_TO_COPY_FOR_CLONE_TABLE: Set[String] = Set(
3339
ClusteringMetadataDomain.domainName)
3440

3541
/**
@@ -91,15 +97,44 @@ object DomainMetadataUtils extends DeltaLogging {
9197

9298
/**
9399
* Generates a new sequence of DomainMetadata to commits for RESTORE TABLE.
94-
* - Source (table to restore to) domains will be copied if they appear in the pre-defined
100+
* - Domains in the toSnapshot will be copied if they appear in the pre-defined
95101
* "copy" list (e.g., table features require some specific domains to be copied).
96-
* - All other domains not in the list are "retained".
102+
* - All other domains not in the list are dropped from the "toSnapshot".
103+
*
104+
* For clustering metadata domain, it overwrites the existing domain metadata in the
105+
* fromSnapshot with the following clustering columns.
106+
* 1. If toSnapshot is not a clustered table or missing domain metadata, use empty clustering
107+
* columns.
108+
* 2. If toSnapshot is a clustered table, use the clustering columns from toSnapshot.
109+
*
110+
* @param toSnapshot The snapshot being restored to, which is referred as "source" table.
111+
* @param fromSnapshot The snapshot being restored from, which is the current state.
97112
*/
98113
def handleDomainMetadataForRestoreTable(
99-
sourceDomainMetadatas: Seq[DomainMetadata]): Seq[DomainMetadata] = {
100-
sourceDomainMetadatas.filter { m =>
114+
toSnapshot: Snapshot,
115+
fromSnapshot: Snapshot): Seq[DomainMetadata] = {
116+
val filteredDomainMetadata = toSnapshot.domainMetadata.filter { m =>
101117
METADATA_DOMAIN_TO_COPY_FOR_RESTORE_TABLE.contains(m.domain)
102118
}
119+
val clusteringColumnsToRestore = ClusteredTableUtils.getClusteringColumnsOptional(toSnapshot)
120+
121+
val isRestoringToClusteredTable =
122+
ClusteredTableUtils.isSupported(toSnapshot.protocol) && clusteringColumnsToRestore.nonEmpty
123+
val clusteringColumns = if (isRestoringToClusteredTable) {
124+
// We overwrite the clustering columns in the fromSnapshot with the clustering columns
125+
// in the toSnapshot.
126+
clusteringColumnsToRestore.get
127+
} else {
128+
// toSnapshot is not a clustered table or missing domain metadata, so we write domain
129+
// metadata with empty clustering columns.
130+
Seq.empty
131+
}
132+
133+
val matchingMetadataDomain =
134+
ClusteredTableUtils.getMatchingMetadataDomain(
135+
clusteringColumns,
136+
fromSnapshot.domainMetadata)
137+
filteredDomainMetadata ++ matchingMetadataDomain.clusteringDomainOpt
103138
}
104139

105140
/**
@@ -112,3 +147,5 @@ object DomainMetadataUtils extends DeltaLogging {
112147
}
113148
}
114149
}
150+
151+
object DomainMetadataUtils extends DomainMetadataUtilsBase

spark/src/main/scala/org/apache/spark/sql/delta/commands/RestoreTableCommand.scala

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import java.sql.Timestamp
2121
import scala.collection.JavaConverters._
2222
import scala.util.{Success, Try}
2323

24-
import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOperations, Snapshot}
24+
import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOperations, DomainMetadataUtils, Snapshot}
2525
import org.apache.spark.sql.delta.actions.{AddFile, DeletionVectorDescriptor, RemoveFile}
2626
import org.apache.spark.sql.delta.catalog.DeltaTableV2
2727
import org.apache.spark.sql.delta.sources.DeltaSQLConf
@@ -205,9 +205,12 @@ case class RestoreTableCommand(sourceTable: DeltaTableV2)
205205
sourceProtocol.merge(targetProtocol)
206206
}
207207

208+
val actions = addActions ++ removeActions ++
209+
DomainMetadataUtils.handleDomainMetadataForRestoreTable(snapshotToRestore, latestSnapshot)
210+
208211
txn.commitLarge(
209212
spark,
210-
addActions ++ removeActions,
213+
actions,
211214
Some(newProtocol),
212215
DeltaOperations.Restore(version, timestamp),
213216
Map.empty,

spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableUtils.scala

Lines changed: 28 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,10 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable
3131
import org.apache.spark.sql.internal.SQLConf
3232
import org.apache.spark.sql.types.{StructField, StructType}
3333

34+
case class MatchingMetadataDomain(
35+
clusteringDomainOpt: Option[DomainMetadata]
36+
)
37+
3438
/**
3539
* Clustered table utility functions.
3640
*/
@@ -146,14 +150,33 @@ trait ClusteredTableUtilsBase extends DeltaLogging {
146150
txn.protocol, txn.metadata, clusterBy)
147151
val clusteringColumns =
148152
clusterBy.columnNames.map(_.toString).map(ClusteringColumn(txn.metadata.schema, _))
149-
Some(createDomainMetadata(clusteringColumns)).toSeq
153+
Seq(createDomainMetadata(clusteringColumns))
150154
}.getOrElse {
151-
if (txn.snapshot.domainMetadata.exists(_.domain == ClusteringMetadataDomain.domainName)) {
152-
Some(createDomainMetadata(Seq.empty)).toSeq
155+
getMatchingMetadataDomain(
156+
clusteringColumns = Seq.empty,
157+
txn.snapshot.domainMetadata).clusteringDomainOpt.toSeq
158+
}
159+
}
160+
161+
/**
162+
* Returns a sequence of [[DomainMetadata]] actions to update the existing domain metadata with
163+
* the given clustering columns.
164+
*
165+
* This is mainly used for REPLACE TABLE and RESTORE TABLE.
166+
*/
167+
def getMatchingMetadataDomain(
168+
clusteringColumns: Seq[ClusteringColumn],
169+
existingDomainMetadata: Seq[DomainMetadata]): MatchingMetadataDomain = {
170+
val clusteringMetadataDomainOpt =
171+
if (existingDomainMetadata.exists(_.domain == ClusteringMetadataDomain.domainName)) {
172+
Some(ClusteringMetadataDomain.fromClusteringColumns(clusteringColumns).toDomainMetadata)
153173
} else {
154-
None.toSeq
174+
None
155175
}
156-
}
176+
177+
MatchingMetadataDomain(
178+
clusteringMetadataDomainOpt
179+
)
157180
}
158181

159182
/**

spark/src/test/scala/org/apache/spark/sql/delta/skipping/ClusteredTableTestUtils.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -152,7 +152,8 @@ trait ClusteredTableTestUtilsBase extends SparkFunSuite with SharedSparkSession
152152
} else {
153153
assertClusterByNotExist()
154154
}
155-
case "WRITE" =>
155+
case "WRITE" | "RESTORE" =>
156+
// These are known operations from our tests that don't have clusterBy.
156157
doAssert(!lastOperationParameters.contains(CLUSTERING_PARAMETER_KEY))
157158
case _ =>
158159
// Other operations are not tested yet. If the test fails here, please check the expected

spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala

Lines changed: 69 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import java.io.File
2121
import com.databricks.spark.util.{Log4jUsageLogger, MetricDefinitions}
2222
import org.apache.spark.sql.delta.skipping.ClusteredTableTestUtils
2323
import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaColumnMappingEnableIdMode, DeltaColumnMappingEnableNameMode, DeltaConfigs, DeltaExcludedBySparkVersionTestMixinShims, DeltaLog, DeltaUnsupportedOperationException}
24+
import org.apache.spark.sql.delta.clustering.ClusteringMetadataDomain
2425
import org.apache.spark.sql.delta.sources.DeltaSQLConf
2526
import org.apache.spark.sql.delta.stats.SkippingEligibleDataType
2627
import org.apache.spark.sql.delta.test.{DeltaColumnMappingSelectedTestMixin, DeltaSQLCommandTest}
@@ -480,7 +481,8 @@ trait ClusteredTableDDLWithColumnMapping
480481
"validate dropping clustering column is not allowed: single clustering column",
481482
"validate dropping clustering column is not allowed: multiple clustering columns",
482483
"validate dropping clustering column is not allowed: clustering column + " +
483-
"non-clustering column"
484+
"non-clustering column",
485+
"validate RESTORE on clustered table"
484486
)
485487

486488
test("validate dropping clustering column is not allowed: single clustering column") {
@@ -825,6 +827,72 @@ trait ClusteredTableDDLSuiteBase
825827
}
826828
}
827829

830+
test("validate RESTORE on clustered table") {
831+
val tableIdentifier = TableIdentifier(testTable)
832+
// Scenario 1: restore clustered table to unclustered version.
833+
withTable(testTable) {
834+
sql(s"CREATE TABLE $testTable (a INT, b STRING) USING delta")
835+
val (_, startingSnapshot) = DeltaLog.forTableWithSnapshot(spark, tableIdentifier)
836+
assert(!ClusteredTableUtils.isSupported(startingSnapshot.protocol))
837+
838+
sql(s"ALTER TABLE $testTable CLUSTER BY (a)")
839+
verifyClusteringColumns(tableIdentifier, "a")
840+
841+
sql(s"RESTORE TABLE $testTable TO VERSION AS OF 0")
842+
val (_, currentSnapshot) = DeltaLog.forTableWithSnapshot(spark, tableIdentifier)
843+
verifyClusteringColumns(tableIdentifier, "")
844+
}
845+
846+
// Scenario 2: restore clustered table to previous clustering columns.
847+
withClusteredTable(testTable, "a INT, b STRING", "a") {
848+
verifyClusteringColumns(tableIdentifier, "a")
849+
850+
sql(s"ALTER TABLE $testTable CLUSTER BY (b)")
851+
verifyClusteringColumns(tableIdentifier, "b")
852+
853+
sql(s"RESTORE TABLE $testTable TO VERSION AS OF 0")
854+
verifyClusteringColumns(tableIdentifier, "a")
855+
}
856+
857+
// Scenario 3: restore from table with clustering columns to non-empty clustering columns
858+
withClusteredTable(testTable, "a int", "a") {
859+
verifyClusteringColumns(tableIdentifier, "a")
860+
861+
sql(s"ALTER TABLE $testTable CLUSTER BY NONE")
862+
verifyClusteringColumns(tableIdentifier, "")
863+
864+
sql(s"RESTORE TABLE $testTable TO VERSION AS OF 0")
865+
verifyClusteringColumns(tableIdentifier, "a")
866+
}
867+
868+
// Scenario 4: restore to start version.
869+
withClusteredTable(testTable, "a int", "a") {
870+
verifyClusteringColumns(tableIdentifier, "a")
871+
872+
sql(s"INSERT INTO $testTable VALUES (1)")
873+
874+
sql(s"RESTORE TABLE $testTable TO VERSION AS OF 0")
875+
verifyClusteringColumns(tableIdentifier, "a")
876+
}
877+
878+
// Scenario 5: restore unclustered table to unclustered table.
879+
withTable(testTable) {
880+
sql(s"CREATE TABLE $testTable (a INT) USING delta")
881+
val (_, startingSnapshot) = DeltaLog.forTableWithSnapshot(spark, tableIdentifier)
882+
assert(!ClusteredTableUtils.isSupported(startingSnapshot.protocol))
883+
assert(!startingSnapshot.domainMetadata.exists(_.domain ==
884+
ClusteringMetadataDomain.domainName))
885+
886+
sql(s"INSERT INTO $testTable VALUES (1)")
887+
888+
sql(s"RESTORE TABLE $testTable TO VERSION AS OF 0").collect
889+
val (_, currentSnapshot) = DeltaLog.forTableWithSnapshot(spark, tableIdentifier)
890+
assert(!ClusteredTableUtils.isSupported(currentSnapshot.protocol))
891+
assert(!currentSnapshot.domainMetadata.exists(_.domain ==
892+
ClusteringMetadataDomain.domainName))
893+
}
894+
}
895+
828896
testSparkMasterOnly("Variant is not supported") {
829897
val e = intercept[DeltaAnalysisException] {
830898
createOrReplaceClusteredTable("CREATE", testTable, "id long, v variant", "v")

0 commit comments

Comments
 (0)