-
Notifications
You must be signed in to change notification settings - Fork 1.7k
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
[Spark] Support external DSV2 catalog in RESTORE command #2033
Closed
gengliangwang
wants to merge
7
commits into
delta-io:master
from
gengliangwang:support3LNameInRestore
Closed
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
d1c8379
workable version
gengliangwang 013f964
fix temp view
gengliangwang 90b616b
add CustomCatalogSuite.scala
gengliangwang be93ff5
move resolveCatalogAndIdentifier to DeltaTableUtils
gengliangwang 32e66aa
fix style
gengliangwang 01e7d79
fix style in DeltaTable.scala
gengliangwang c0baf09
remove unnecessary changes; add one comment
gengliangwang 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 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
This file contains 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
This file contains 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
This file contains 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
142 changes: 142 additions & 0 deletions
142
spark/src/test/scala/org/apache/spark/sql/delta/CustomCatalogSuite.scala
This file contains 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,142 @@ | ||
/* | ||
* Copyright (2023) 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 java.util | ||
import org.apache.spark.sql.connector.catalog._ | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.sql.connector.expressions.Transform | ||
import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
import io.delta.tables.DeltaTable | ||
import org.apache.hadoop.fs.{FileSystem, Path} | ||
|
||
import java.nio.file.{Files, Paths} | ||
import org.apache.spark.sql.delta.DeltaLog | ||
import org.apache.spark.sql.{QueryTest, SparkSession} | ||
import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog, TableChange} | ||
import org.apache.spark.sql.connector.expressions.Transform | ||
import org.apache.spark.sql.delta.catalog.{DeltaCatalog, DeltaTableV2} | ||
import org.apache.spark.sql.delta.test.DeltaSQLCommandTest | ||
import org.apache.spark.sql.test.SharedSparkSession | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
import org.apache.spark.util.Utils | ||
import org.apache.spark.SparkConf | ||
|
||
import java.io.{File, IOException} | ||
|
||
|
||
class CustomCatalogSuite extends QueryTest with SharedSparkSession | ||
with DeltaSQLCommandTest { | ||
|
||
override def sparkConf: SparkConf = | ||
super.sparkConf.set("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) | ||
|
||
test("RESTORE a table from DummyCatalog") { | ||
val tableName = "table1" | ||
withTable(tableName) { | ||
sql("SET CATALOG dummy") | ||
val dummyCatalog = | ||
spark.sessionState.catalogManager.catalog("dummy").asInstanceOf[DummyCatalog] | ||
val tablePath = dummyCatalog.getTablePath(tableName) | ||
sql(f"CREATE TABLE $tableName (id bigint) USING delta") | ||
// Insert some data into the table in the dummy catalog. | ||
// To make it simple, here we insert data directly into the table path. | ||
sql(f"INSERT INTO delta.`$tablePath` VALUES (0)") | ||
sql(f"INSERT INTO delta.`$tablePath` VALUES (1)") | ||
sql(f"RESTORE TABLE $tableName VERSION AS OF 0") | ||
checkAnswer(spark.table(tableName), Nil) | ||
sql(f"RESTORE TABLE $tableName VERSION AS OF 1") | ||
checkAnswer(spark.table(tableName), spark.range(1).toDF()) | ||
// Test 3-part identifier | ||
sql(f"RESTORE TABLE dummy.default.$tableName VERSION AS OF 2") | ||
checkAnswer(spark.table(tableName), spark.range(2).toDF()) | ||
|
||
sql("SET CATALOG spark_catalog") | ||
// Test 3-part identifier when the current catalog is the default catalog | ||
sql(f"RESTORE TABLE dummy.default.$tableName VERSION AS OF 1") | ||
checkAnswer(spark.table(f"dummy.default.$tableName"), spark.range(1).toDF()) | ||
} | ||
} | ||
} | ||
|
||
class DummyCatalog extends TableCatalog { | ||
private val spark: SparkSession = SparkSession.active | ||
private val tempDir: Path = new Path(Utils.createTempDir().getAbsolutePath) | ||
// scalastyle:off deltahadoopconfiguration | ||
private val fs: FileSystem = | ||
tempDir.getFileSystem(spark.sessionState.newHadoopConf()) | ||
// scalastyle:on deltahadoopconfiguration | ||
|
||
override def name: String = "dummy" | ||
|
||
def getTablePath(tableName: String): Path = { | ||
new Path(tempDir, tableName) | ||
} | ||
override def defaultNamespace(): Array[String] = Array("default") | ||
|
||
override def listTables(namespace: Array[String]): Array[Identifier] = { | ||
val status = fs.listStatus(tempDir) | ||
status.filter(_.isDirectory).map { dir => | ||
Identifier.of(namespace, dir.getPath.getName) | ||
} | ||
} | ||
|
||
override def tableExists(ident: Identifier): Boolean = { | ||
val tablePath = getTablePath(ident.name()) | ||
fs.exists(tablePath) | ||
} | ||
override def loadTable(ident: Identifier): Table = { | ||
val tablePath = getTablePath(ident.name()) | ||
DeltaTableV2(spark, tablePath) | ||
} | ||
|
||
override def createTable( | ||
ident: Identifier, | ||
schema: StructType, | ||
partitions: Array[Transform], | ||
properties: java.util.Map[String, String]): Table = { | ||
val tablePath = getTablePath(ident.name()) | ||
// Create an empty Delta table on the tablePath | ||
spark.range(0).write.format("delta").save(tablePath.toString) | ||
DeltaTableV2(spark, tablePath) | ||
} | ||
|
||
override def alterTable(ident: Identifier, changes: TableChange*): Table = { | ||
throw new UnsupportedOperationException("Alter table operation is not supported.") | ||
} | ||
|
||
override def dropTable(ident: Identifier): Boolean = { | ||
val tablePath = getTablePath(ident.name()) | ||
try { | ||
fs.delete(tablePath, true) | ||
true | ||
} catch { | ||
case _: Exception => false | ||
} | ||
} | ||
|
||
override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = { | ||
throw new UnsupportedOperationException("Rename table operation is not supported.") | ||
} | ||
|
||
override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { | ||
// Initialize tempDir here | ||
if (!fs.exists(tempDir)) { | ||
fs.mkdirs(tempDir) | ||
} | ||
} | ||
} |
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shouldn't this use the passed-in schema? Otherwise it takes a DDL later to update it with info we already had...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Alternatively, we can just create the DTV2 and call it good -- Delta knows how to handle the new/empty/missing directory case, tho it won't let you read such tables. Which comes back to the first comment -- if the table needs to be readable after this, it needs the correct schema, no?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since it is a dummy catalog, I try to make the schema fixed as "id: long"
Could you show me some details of how to use it?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
DeltaTableV2(spark, tablePath.toString)
should suffice? See DeltaTableV2.scala