-
Notifications
You must be signed in to change notification settings - Fork 2.6k
Spark: Implement RollbackToSnapshotProcedure #1759
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
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
55 changes: 55 additions & 0 deletions
55
...extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
/* | ||
* 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.iceberg.spark.extensions; | ||
|
||
import java.util.Map; | ||
import org.apache.iceberg.hive.HiveCatalog; | ||
import org.apache.iceberg.hive.TestHiveMetastore; | ||
import org.apache.iceberg.spark.SparkCatalogTestBase; | ||
import org.apache.iceberg.spark.SparkTestBase; | ||
import org.apache.spark.sql.SparkSession; | ||
import org.apache.spark.sql.internal.SQLConf; | ||
import org.junit.BeforeClass; | ||
|
||
import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; | ||
|
||
public abstract class SparkExtensionsTestBase extends SparkCatalogTestBase { | ||
|
||
public SparkExtensionsTestBase(String catalogName, String implementation, Map<String, String> config) { | ||
super(catalogName, implementation, config); | ||
} | ||
|
||
@BeforeClass | ||
public static void startMetastoreAndSpark() { | ||
SparkTestBase.metastore = new TestHiveMetastore(); | ||
metastore.start(); | ||
SparkTestBase.hiveConf = metastore.hiveConf(); | ||
|
||
SparkTestBase.spark = SparkSession.builder() | ||
.master("local[2]") | ||
.config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") | ||
.config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) | ||
.config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) | ||
.enableHiveSupport() | ||
.getOrCreate(); | ||
|
||
SparkTestBase.catalog = new HiveCatalog(spark.sessionState().newHadoopConf()); | ||
} | ||
} |
272 changes: 272 additions & 0 deletions
272
...ns/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,272 @@ | ||
/* | ||
* 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.iceberg.spark.extensions; | ||
|
||
import java.util.List; | ||
import java.util.Map; | ||
import org.apache.iceberg.AssertHelpers; | ||
import org.apache.iceberg.Snapshot; | ||
import org.apache.iceberg.Table; | ||
import org.apache.iceberg.catalog.Namespace; | ||
import org.apache.iceberg.exceptions.ValidationException; | ||
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; | ||
import org.apache.spark.sql.AnalysisException; | ||
import org.apache.spark.sql.Dataset; | ||
import org.apache.spark.sql.Row; | ||
import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; | ||
import org.junit.After; | ||
import org.junit.Assume; | ||
import org.junit.Test; | ||
|
||
public class TestRollbackToSnapshotProcedure extends SparkExtensionsTestBase { | ||
|
||
public TestRollbackToSnapshotProcedure(String catalogName, String implementation, Map<String, String> config) { | ||
super(catalogName, implementation, config); | ||
} | ||
|
||
@After | ||
public void removeTables() { | ||
sql("DROP TABLE IF EXISTS %s", tableName); | ||
} | ||
|
||
@Test | ||
public void testRollbackToSnapshotUsingPositionalArgs() { | ||
sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); | ||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
Table table = validationCatalog.loadTable(tableIdent); | ||
Snapshot firstSnapshot = table.currentSnapshot(); | ||
|
||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
assertEquals("Should have expected rows", | ||
ImmutableList.of(row(1L, "a"), row(1L, "a")), | ||
sql("SELECT * FROM %s ORDER BY id", tableName)); | ||
|
||
table.refresh(); | ||
|
||
Snapshot secondSnapshot = table.currentSnapshot(); | ||
|
||
List<Object[]> output = sql( | ||
"CALL %s.system.rollback_to_snapshot('%s', '%s', %dL)", | ||
catalogName, tableIdent.namespace(), tableIdent.name(), firstSnapshot.snapshotId()); | ||
|
||
assertEquals("Procedure output must match", | ||
ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), | ||
output); | ||
|
||
assertEquals("Rollback must be successful", | ||
ImmutableList.of(row(1L, "a")), | ||
sql("SELECT * FROM %s ORDER BY id", tableName)); | ||
} | ||
|
||
@Test | ||
public void testRollbackToSnapshotUsingNamedArgs() { | ||
sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); | ||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
Table table = validationCatalog.loadTable(tableIdent); | ||
Snapshot firstSnapshot = table.currentSnapshot(); | ||
|
||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
assertEquals("Should have expected rows", | ||
ImmutableList.of(row(1L, "a"), row(1L, "a")), | ||
sql("SELECT * FROM %s ORDER BY id", tableName)); | ||
|
||
table.refresh(); | ||
|
||
Snapshot secondSnapshot = table.currentSnapshot(); | ||
|
||
List<Object[]> output = sql( | ||
"CALL %s.system.rollback_to_snapshot(snapshot_id => %dL, namespace => '%s', table => '%s')", | ||
catalogName, firstSnapshot.snapshotId(), tableIdent.namespace(), tableIdent.name()); | ||
|
||
assertEquals("Procedure output must match", | ||
ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), | ||
output); | ||
|
||
assertEquals("Rollback must be successful", | ||
ImmutableList.of(row(1L, "a")), | ||
sql("SELECT * FROM %s ORDER BY id", tableName)); | ||
} | ||
|
||
@Test | ||
public void testRollbackToSnapshotRefreshesRelationCache() { | ||
sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); | ||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
Table table = validationCatalog.loadTable(tableIdent); | ||
Snapshot firstSnapshot = table.currentSnapshot(); | ||
|
||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
table.refresh(); | ||
|
||
Snapshot secondSnapshot = table.currentSnapshot(); | ||
|
||
Dataset<Row> query = spark.sql("SELECT * FROM " + tableName + " WHERE id = 1"); | ||
query.createOrReplaceTempView("tmp"); | ||
|
||
spark.sql("CACHE TABLE tmp"); | ||
|
||
assertEquals("View should have expected rows", | ||
ImmutableList.of(row(1L, "a"), row(1L, "a")), | ||
sql("SELECT * FROM tmp")); | ||
|
||
List<Object[]> output = sql( | ||
"CALL %s.system.rollback_to_snapshot(namespace => '%s', table => '%s', snapshot_id => %dL)", | ||
catalogName, tableIdent.namespace(), tableIdent.name(), firstSnapshot.snapshotId()); | ||
|
||
assertEquals("Procedure output must match", | ||
ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), | ||
output); | ||
|
||
assertEquals("View cache must be invalidated", | ||
ImmutableList.of(row(1L, "a")), | ||
sql("SELECT * FROM tmp")); | ||
|
||
sql("UNCACHE TABLE tmp"); | ||
} | ||
|
||
@Test | ||
public void testRollbackToSnapshotWithQuotedIdentifiers() { | ||
sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); | ||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
Table table = validationCatalog.loadTable(tableIdent); | ||
Snapshot firstSnapshot = table.currentSnapshot(); | ||
|
||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
assertEquals("Should have expected rows", | ||
ImmutableList.of(row(1L, "a"), row(1L, "a")), | ||
sql("SELECT * FROM %s ORDER BY id", tableName)); | ||
|
||
table.refresh(); | ||
|
||
Snapshot secondSnapshot = table.currentSnapshot(); | ||
|
||
StringBuilder quotedNamespaceBuilder = new StringBuilder(); | ||
for (String level : tableIdent.namespace().levels()) { | ||
quotedNamespaceBuilder.append("`"); | ||
quotedNamespaceBuilder.append(level); | ||
quotedNamespaceBuilder.append("`"); | ||
} | ||
String quotedNamespace = quotedNamespaceBuilder.toString(); | ||
|
||
List<Object[]> output = sql( | ||
"CALL %s.system.rollback_to_snapshot('%s', '`%s`', %d)", | ||
aokolnychyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
catalogName, quotedNamespace, tableIdent.name(), firstSnapshot.snapshotId()); | ||
|
||
assertEquals("Procedure output must match", | ||
ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), | ||
output); | ||
|
||
assertEquals("Rollback must be successful", | ||
ImmutableList.of(row(1L, "a")), | ||
sql("SELECT * FROM %s ORDER BY id", tableName)); | ||
} | ||
|
||
@Test | ||
public void testRollbackToSnapshotWithoutExplicitCatalog() { | ||
Assume.assumeTrue("Working only with the session catalog", "spark_catalog".equals(catalogName)); | ||
|
||
sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); | ||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
Table table = validationCatalog.loadTable(tableIdent); | ||
Snapshot firstSnapshot = table.currentSnapshot(); | ||
|
||
sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); | ||
|
||
assertEquals("Should have expected rows", | ||
ImmutableList.of(row(1L, "a"), row(1L, "a")), | ||
sql("SELECT * FROM %s ORDER BY id", tableName)); | ||
|
||
table.refresh(); | ||
|
||
Snapshot secondSnapshot = table.currentSnapshot(); | ||
|
||
// use camel case intentionally to test case sensitivity | ||
List<Object[]> output = sql( | ||
"CALL SyStEm.rOLlBaCk_to_SnApShOt('%s', '%s', %dL)", | ||
tableIdent.namespace(), tableIdent.name(), firstSnapshot.snapshotId()); | ||
|
||
assertEquals("Procedure output must match", | ||
ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), | ||
output); | ||
|
||
assertEquals("Rollback must be successful", | ||
ImmutableList.of(row(1L, "a")), | ||
sql("SELECT * FROM %s ORDER BY id", tableName)); | ||
} | ||
|
||
@Test | ||
public void testRollbackToInvalidSnapshot() { | ||
sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); | ||
|
||
Namespace namespace = tableIdent.namespace(); | ||
String tableName = tableIdent.name(); | ||
|
||
AssertHelpers.assertThrows("Should reject invalid snapshot id", | ||
ValidationException.class, "Cannot roll back to unknown snapshot id", | ||
() -> sql("CALL %s.system.rollback_to_snapshot('%s', '%s', -1L)", catalogName, namespace, tableName)); | ||
} | ||
|
||
@Test | ||
public void testInvalidRollbackToSnapshotCases() { | ||
AssertHelpers.assertThrows("Should not allow mixed args", | ||
AnalysisException.class, "Named and positional arguments cannot be mixed", | ||
() -> sql("CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", catalogName)); | ||
|
||
AssertHelpers.assertThrows("Should not resolve procedures in arbitrary namespaces", | ||
NoSuchProcedureException.class, "not found", | ||
() -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)); | ||
|
||
AssertHelpers.assertThrows("Should reject calls without all required args", | ||
AnalysisException.class, "Missing required parameters", | ||
() -> sql("CALL %s.system.rollback_to_snapshot('n', 't')", catalogName)); | ||
|
||
AssertHelpers.assertThrows("Should reject calls without all required args", | ||
AnalysisException.class, "Missing required parameters", | ||
() -> sql("CALL %s.system.rollback_to_snapshot('n', 1L)", catalogName)); | ||
|
||
AssertHelpers.assertThrows("Should reject calls without all required args", | ||
AnalysisException.class, "Missing required parameters", | ||
() -> sql("CALL %s.system.rollback_to_snapshot(namespace => 'n', snapshot_id => 1L)", catalogName)); | ||
|
||
AssertHelpers.assertThrows("Should reject calls without all required args", | ||
AnalysisException.class, "Missing required parameters", | ||
() -> sql("CALL %s.system.rollback_to_snapshot(table => 't', snapshot_id => 1L)", catalogName)); | ||
|
||
AssertHelpers.assertThrows("Should reject calls with invalid arg types", | ||
AnalysisException.class, "Wrong arg type for snapshot_id: expected LongType", | ||
() -> sql("CALL %s.system.rollback_to_snapshot('n', 't', 2.2)", catalogName)); | ||
|
||
AssertHelpers.assertThrows("Should reject empty namespace", | ||
IllegalArgumentException.class, "Namespace cannot be empty", | ||
() -> sql("CALL %s.system.rollback_to_snapshot('', 't', 1L)", catalogName)); | ||
|
||
AssertHelpers.assertThrows("Should reject empty table name", | ||
IllegalArgumentException.class, "Table name cannot be empty", | ||
() -> sql("CALL %s.system.rollback_to_snapshot('n', '', 1L)", catalogName)); | ||
} | ||
} |
48 changes: 48 additions & 0 deletions
48
spark3/src/main/java/org/apache/iceberg/spark/BaseCatalog.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,48 @@ | ||
/* | ||
* 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.iceberg.spark; | ||
|
||
import org.apache.iceberg.spark.procedures.SparkProcedures; | ||
import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; | ||
import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; | ||
import org.apache.spark.sql.connector.catalog.Identifier; | ||
import org.apache.spark.sql.connector.catalog.StagingTableCatalog; | ||
import org.apache.spark.sql.connector.catalog.SupportsNamespaces; | ||
import org.apache.spark.sql.connector.iceberg.catalog.Procedure; | ||
import org.apache.spark.sql.connector.iceberg.catalog.ProcedureCatalog; | ||
|
||
abstract class BaseCatalog implements StagingTableCatalog, ProcedureCatalog, SupportsNamespaces { | ||
|
||
@Override | ||
public Procedure loadProcedure(Identifier ident) throws NoSuchProcedureException { | ||
String[] namespace = ident.namespace(); | ||
String name = ident.name(); | ||
|
||
// namespace resolution is case insensitive until we have a way to configure case sensitivity in catalogs | ||
if (namespace.length == 1 && namespace[0].equalsIgnoreCase("system")) { | ||
ProcedureBuilder builder = SparkProcedures.newBuilder(name); | ||
if (builder != null) { | ||
return builder.withTableCatalog(this).build(); | ||
} | ||
} | ||
|
||
throw new NoSuchProcedureException(ident); | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.