alexeykudinkin commented on code in PR #6264:
URL: https://github.com/apache/hudi/pull/6264#discussion_r940735559


##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieCatalogAndIdentifier.scala:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.connector.catalog
+
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+
+object HoodieCatalogAndIdentifier {
+
+  import 
org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
+
+  private val globalTempDB = 
SQLConf.get.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE)
+
+  def parse(catalogManager: CatalogManager, nameParts: Seq[String])
+      : Option[(CatalogPlugin, Identifier)] = {
+    assert(nameParts.nonEmpty)
+    val currentCatalog = catalogManager.currentCatalog
+    if (nameParts.length == 1) {
+      Some((currentCatalog, Identifier.of(catalogManager.currentNamespace, 
nameParts.head)))
+    } else if (nameParts.head.equalsIgnoreCase(globalTempDB)) {
+      // Conceptually global temp views are in a special reserved catalog. 
However, the v2 catalog

Review Comment:
   Was this borrowed from somewhere?



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystPlanUtils.scala:
##########
@@ -52,8 +57,56 @@ abstract class HoodieSpark3CatalystPlanUtils extends 
HoodieCatalystPlansUtils {
     }
   }
 
-  override def toTableIdentifier(relation: UnresolvedRelation): 
TableIdentifier = {
-    relation.multipartIdentifier.asTableIdentifier
+  override def resolve(spark: SparkSession, relation: UnresolvedRelation): 
Option[CatalogTable] = {
+    val catalogManager = spark.sessionState.catalogManager
+    val nameParts = relation.multipartIdentifier
+    val expandedNameParts = expandIdentifier(spark, nameParts)
+    HoodieCatalogAndIdentifier.parse(catalogManager, expandedNameParts) match {
+      case Some((catalog, ident)) =>
+        CatalogV2Util.loadTable(catalog, ident) match {
+          case Some(table) =>
+            table match {
+              case v1Table: V1Table =>
+                Some(v1Table.v1Table)
+              case withFallback: V2TableWithV1Fallback =>
+                Some(withFallback.v1Table)
+              case _ =>
+                logWarning(s"It's not a hoodie table: $table")
+                None
+            }
+          case _ =>
+            logWarning(s"Can not load this catalog and identifier: 
${catalog.name()}, $ident")
+            None
+        }
+      case _ =>
+        logWarning(s"Can not parse this name parts: 
${expandedNameParts.mkString(",")}")

Review Comment:
   nit: `${expandedNameParts.mkString("[", ",", "]")}`



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystPlanUtils.scala:
##########
@@ -52,8 +57,56 @@ abstract class HoodieSpark3CatalystPlanUtils extends 
HoodieCatalystPlansUtils {
     }
   }
 
-  override def toTableIdentifier(relation: UnresolvedRelation): 
TableIdentifier = {
-    relation.multipartIdentifier.asTableIdentifier
+  override def resolve(spark: SparkSession, relation: UnresolvedRelation): 
Option[CatalogTable] = {

Review Comment:
   Can you please elaborate why we need to have this custom resolution rule 
even though there's Spark's default `ResolveRelations` one?



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystPlanUtils.scala:
##########
@@ -52,8 +57,56 @@ abstract class HoodieSpark3CatalystPlanUtils extends 
HoodieCatalystPlansUtils {
     }
   }
 
-  override def toTableIdentifier(relation: UnresolvedRelation): 
TableIdentifier = {
-    relation.multipartIdentifier.asTableIdentifier
+  override def resolve(spark: SparkSession, relation: UnresolvedRelation): 
Option[CatalogTable] = {
+    val catalogManager = spark.sessionState.catalogManager
+    val nameParts = relation.multipartIdentifier
+    val expandedNameParts = expandIdentifier(spark, nameParts)
+    HoodieCatalogAndIdentifier.parse(catalogManager, expandedNameParts) match {
+      case Some((catalog, ident)) =>
+        CatalogV2Util.loadTable(catalog, ident) match {
+          case Some(table) =>
+            table match {
+              case v1Table: V1Table =>
+                Some(v1Table.v1Table)
+              case withFallback: V2TableWithV1Fallback =>
+                Some(withFallback.v1Table)
+              case _ =>
+                logWarning(s"It's not a hoodie table: $table")

Review Comment:
   Warning is misleading: we don't check anywhere whether this is a Hudi table 
or not, we will hit this if there's gonna be another type of table introduced



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3Catalog.scala:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.hudi
+
+import java.sql.{Connection, DriverManager}
+import java.util.Properties
+
+import org.apache.hudi.HoodieSparkUtils
+
+import org.apache.spark.SparkConf
+import org.apache.spark.util.Utils
+
+import java.io.File
+
+class TestSpark3Catalog extends HoodieSparkSqlTestBase {
+
+  val tempDir: File = Utils.createTempDir()
+  val url = 
s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass"
+
+  override def sparkConf(): SparkConf = {
+    val sparkConf = super.sparkConf()
+    if (HoodieSparkUtils.gteqSpark3_1) {

Review Comment:
   But how would it mess it up? We're only running our test for Spark 3.x, 
right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to