You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2019/03/22 01:05:15 UTC

[spark] branch master updated: [SPARK-26946][SQL] Identifiers for multi-catalog

This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 80565ce  [SPARK-26946][SQL] Identifiers for multi-catalog
80565ce is described below

commit 80565ce253c6efc0d5b5966122d3e81262e5c82f
Author: John Zhuge <jz...@apache.org>
AuthorDate: Thu Mar 21 18:04:50 2019 -0700

    [SPARK-26946][SQL] Identifiers for multi-catalog
    
    ## What changes were proposed in this pull request?
    
    - Support N-part identifier in SQL
    - N-part identifier extractor in Analyzer
    
    ## How was this patch tested?
    
    - A new unit test suite ResolveMultipartRelationSuite
    - CatalogLoadingSuite
    
    rblue cloud-fan mccheah
    
    Closes #23848 from jzhuge/SPARK-26946.
    
    Authored-by: John Zhuge <jz...@apache.org>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../apache/spark/sql/catalyst/parser/SqlBase.g4    |  8 ++
 .../org/apache/spark/sql/catalog/v2/Catalogs.java  |  8 +-
 .../apache/spark/sql/catalog/v2/Identifier.java    | 41 +++++++++
 .../spark/sql/catalog/v2/IdentifierImpl.java       | 45 ++++++++++
 .../sql/catalog/v2/CatalogNotFoundException.scala  | 28 ++++++
 .../spark/sql/catalog/v2/LookupCatalog.scala       | 74 ++++++++++++++++
 .../spark/sql/catalyst/analysis/Analyzer.scala     | 11 ++-
 .../spark/sql/catalyst/parser/AstBuilder.scala     | 13 +++
 .../spark/sql/catalyst/parser/ParseDriver.scala    |  7 ++
 .../sql/catalyst/parser/ParserInterface.scala      |  6 ++
 .../spark/sql/catalog/v2/CatalogLoadingSuite.java  |  3 +-
 .../v2/ResolveMultipartIdentifierSuite.scala       | 99 ++++++++++++++++++++++
 .../spark/sql/SparkSessionExtensionSuite.scala     |  3 +
 13 files changed, 340 insertions(+), 6 deletions(-)

diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index 8bd7df5..1f7da19 100644
--- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -63,6 +63,10 @@ singleTableIdentifier
     : tableIdentifier EOF
     ;
 
+singleMultipartIdentifier
+    : multipartIdentifier EOF
+    ;
+
 singleFunctionIdentifier
     : functionIdentifier EOF
     ;
@@ -554,6 +558,10 @@ rowFormat
       (NULL DEFINED AS nullDefinedAs=STRING)?                                       #rowFormatDelimited
     ;
 
+multipartIdentifier
+    : parts+=identifier ('.' parts+=identifier)*
+    ;
+
 tableIdentifier
     : (db=identifier '.')? table=identifier
     ;
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java
index aa4cbfc..851a6a9 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java
@@ -44,12 +44,14 @@ public class Catalogs {
    * @param name a String catalog name
    * @param conf a SQLConf
    * @return an initialized CatalogPlugin
-   * @throws SparkException If the plugin class cannot be found or instantiated
+   * @throws CatalogNotFoundException if the plugin class cannot be found
+   * @throws SparkException if the plugin class cannot be instantiated
    */
-  public static CatalogPlugin load(String name, SQLConf conf) throws SparkException {
+  public static CatalogPlugin load(String name, SQLConf conf)
+      throws CatalogNotFoundException, SparkException {
     String pluginClassName = conf.getConfString("spark.sql.catalog." + name, null);
     if (pluginClassName == null) {
-      throw new SparkException(String.format(
+      throw new CatalogNotFoundException(String.format(
           "Catalog '%s' plugin class not found: spark.sql.catalog.%s is not defined", name, name));
     }
 
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java
new file mode 100644
index 0000000..3e697c1
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java
@@ -0,0 +1,41 @@
+/*
+ * 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.catalog.v2;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ * Identifies an object in a catalog.
+ */
+@Experimental
+public interface Identifier {
+
+  static Identifier of(String[] namespace, String name) {
+    return new IdentifierImpl(namespace, name);
+  }
+
+  /**
+   * @return the namespace in the catalog
+   */
+  String[] namespace();
+
+  /**
+   * @return the object name
+   */
+  String name();
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java
new file mode 100644
index 0000000..8874faa
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java
@@ -0,0 +1,45 @@
+/*
+ * 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.catalog.v2;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ *  An {@link Identifier} implementation.
+ */
+@Experimental
+class IdentifierImpl implements Identifier {
+
+  private String[] namespace;
+  private String name;
+
+  IdentifierImpl(String[] namespace, String name) {
+    this.namespace = namespace;
+    this.name = name;
+  }
+
+  @Override
+  public String[] namespace() {
+    return namespace;
+  }
+
+  @Override
+  public String name() {
+    return name;
+  }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala
new file mode 100644
index 0000000..86de1c92
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala
@@ -0,0 +1,28 @@
+/*
+ * 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.catalog.v2
+
+import org.apache.spark.SparkException
+import org.apache.spark.annotation.Experimental
+
+@Experimental
+class CatalogNotFoundException(message: String, cause: Throwable)
+  extends SparkException(message, cause) {
+
+  def this(message: String) = this(message, null)
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala
new file mode 100644
index 0000000..932d320
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala
@@ -0,0 +1,74 @@
+/*
+ * 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.catalog.v2
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.TableIdentifier
+
+/**
+ * A trait to encapsulate catalog lookup function and helpful extractors.
+ */
+@Experimental
+trait LookupCatalog {
+
+  def lookupCatalog: Option[(String) => CatalogPlugin] = None
+
+  type CatalogObjectIdentifier = (Option[CatalogPlugin], Identifier)
+
+  /**
+   * Extract catalog plugin and identifier from a multi-part identifier.
+   */
+  object CatalogObjectIdentifier {
+    def unapply(parts: Seq[String]): Option[CatalogObjectIdentifier] = lookupCatalog.map { lookup =>
+      parts match {
+        case Seq(name) =>
+          (None, Identifier.of(Array.empty, name))
+        case Seq(catalogName, tail @ _*) =>
+          try {
+            val catalog = lookup(catalogName)
+            (Some(catalog), Identifier.of(tail.init.toArray, tail.last))
+          } catch {
+            case _: CatalogNotFoundException =>
+              (None, Identifier.of(parts.init.toArray, parts.last))
+          }
+      }
+    }
+  }
+
+  /**
+   * Extract legacy table identifier from a multi-part identifier.
+   *
+   * For legacy support only. Please use
+   * [[org.apache.spark.sql.catalog.v2.LookupCatalog.CatalogObjectIdentifier]] in DSv2 code paths.
+   */
+  object AsTableIdentifier {
+    def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match {
+      case CatalogObjectIdentifier(None, ident) =>
+        ident.namespace match {
+          case Array() =>
+            Some(TableIdentifier(ident.name))
+          case Array(database) =>
+            Some(TableIdentifier(ident.name, Some(database)))
+          case _ =>
+            None
+        }
+      case _ =>
+        None
+    }
+  }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index e4cf43d..e216234 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -24,6 +24,7 @@ import scala.collection.mutable.ArrayBuffer
 import scala.util.Random
 
 import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalog.v2.{CatalogPlugin, LookupCatalog}
 import org.apache.spark.sql.catalyst._
 import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.encoders.OuterScopes
@@ -95,13 +96,19 @@ object AnalysisContext {
 class Analyzer(
     catalog: SessionCatalog,
     conf: SQLConf,
-    maxIterations: Int)
-  extends RuleExecutor[LogicalPlan] with CheckAnalysis {
+    maxIterations: Int,
+    override val lookupCatalog: Option[(String) => CatalogPlugin] = None)
+  extends RuleExecutor[LogicalPlan] with CheckAnalysis with LookupCatalog {
 
   def this(catalog: SessionCatalog, conf: SQLConf) = {
     this(catalog, conf, conf.optimizerMaxIterations)
   }
 
+  def this(lookupCatalog: Option[(String) => CatalogPlugin], catalog: SessionCatalog,
+      conf: SQLConf) = {
+    this(catalog, conf, conf.optimizerMaxIterations, lookupCatalog)
+  }
+
   def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = {
     AnalysisHelper.markInAnalyzer {
       val analyzed = executeAndTrack(plan, tracker)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 4cce1c1..b1d6be5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -86,6 +86,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
     visitFunctionIdentifier(ctx.functionIdentifier)
   }
 
+  override def visitSingleMultipartIdentifier(
+      ctx: SingleMultipartIdentifierContext): Seq[String] = withOrigin(ctx) {
+    visitMultipartIdentifier(ctx.multipartIdentifier)
+  }
+
   override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
     visitSparkDataType(ctx.dataType)
   }
@@ -957,6 +962,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
     FunctionIdentifier(ctx.function.getText, Option(ctx.db).map(_.getText))
   }
 
+  /**
+   * Create a multi-part identifier.
+   */
+  override def visitMultipartIdentifier(
+      ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) {
+    ctx.parts.asScala.map(_.getText)
+  }
+
   /* ********************************************************************************************
    * Expression parsing
    * ******************************************************************************************** */
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
index 1d5de0a..d5c7bb7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
@@ -57,6 +57,13 @@ abstract class AbstractSqlParser extends ParserInterface with Logging {
     }
   }
 
+  /** Creates a multi-part identifier for a given SQL string */
+  override def parseMultipartIdentifier(sqlText: String): Seq[String] = {
+    parse(sqlText) { parser =>
+      astBuilder.visitSingleMultipartIdentifier(parser.singleMultipartIdentifier())
+    }
+  }
+
   /**
    * Creates StructType for a given SQL string, which is a comma separated list of field
    * definitions which will preserve the correct Hive metadata.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
index 75240d2..77e357a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
@@ -53,6 +53,12 @@ trait ParserInterface {
   def parseFunctionIdentifier(sqlText: String): FunctionIdentifier
 
   /**
+   * Parse a string to a multi-part identifier.
+   */
+  @throws[ParseException]("Text cannot be parsed to a multi-part identifier")
+  def parseMultipartIdentifier(sqlText: String): Seq[String]
+
+  /**
    * Parse a string to a [[StructType]]. The passed SQL string should be a comma separated list
    * of field definitions which will preserve the correct Hive metadata.
    */
diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java
index 2f55da8..326b12f 100644
--- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java
+++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java
@@ -66,7 +66,8 @@ public class CatalogLoadingSuite {
   public void testLoadWithoutConfig() {
     SQLConf conf = new SQLConf();
 
-    SparkException exc = intercept(SparkException.class, () -> Catalogs.load("missing", conf));
+    SparkException exc = intercept(CatalogNotFoundException.class,
+        () -> Catalogs.load("missing", conf));
 
     Assert.assertTrue("Should complain that implementation is not configured",
         exc.getMessage()
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/ResolveMultipartIdentifierSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/ResolveMultipartIdentifierSuite.scala
new file mode 100644
index 0000000..0f2d67e
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/ResolveMultipartIdentifierSuite.scala
@@ -0,0 +1,99 @@
+/*
+ * 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.catalog.v2
+
+import org.scalatest.Matchers._
+
+import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+private class TestCatalogPlugin(override val name: String) extends CatalogPlugin {
+
+  override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = Unit
+}
+
+class ResolveMultipartIdentifierSuite extends AnalysisTest {
+  import CatalystSqlParser._
+
+  private val analyzer = makeAnalyzer(caseSensitive = false)
+
+  private val catalogs = Seq("prod", "test").map(name => name -> new TestCatalogPlugin(name)).toMap
+
+  private def lookupCatalog(catalog: String): CatalogPlugin =
+    catalogs.getOrElse(catalog, throw new CatalogNotFoundException("Not found"))
+
+  private def makeAnalyzer(caseSensitive: Boolean) = {
+    val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> caseSensitive)
+    new Analyzer(Some(lookupCatalog _), null, conf)
+  }
+
+  override protected def getAnalyzer(caseSensitive: Boolean) = analyzer
+
+  private def checkResolution(sqlText: String, expectedCatalog: Option[CatalogPlugin],
+      expectedNamespace: Array[String], expectedName: String): Unit = {
+
+    import analyzer.CatalogObjectIdentifier
+    val CatalogObjectIdentifier(catalog, ident) = parseMultipartIdentifier(sqlText)
+    catalog shouldEqual expectedCatalog
+    ident.namespace shouldEqual expectedNamespace
+    ident.name shouldEqual expectedName
+  }
+
+  private def checkTableResolution(sqlText: String,
+      expectedIdent: Option[TableIdentifier]): Unit = {
+
+    import analyzer.AsTableIdentifier
+    parseMultipartIdentifier(sqlText) match {
+      case AsTableIdentifier(ident) =>
+        assert(Some(ident) === expectedIdent)
+      case _ =>
+        assert(None === expectedIdent)
+    }
+  }
+
+  test("resolve multipart identifier") {
+    checkResolution("tbl", None, Array.empty, "tbl")
+    checkResolution("db.tbl", None, Array("db"), "tbl")
+    checkResolution("prod.func", catalogs.get("prod"), Array.empty, "func")
+    checkResolution("ns1.ns2.tbl", None, Array("ns1", "ns2"), "tbl")
+    checkResolution("prod.db.tbl", catalogs.get("prod"), Array("db"), "tbl")
+    checkResolution("test.db.tbl", catalogs.get("test"), Array("db"), "tbl")
+    checkResolution("test.ns1.ns2.ns3.tbl",
+      catalogs.get("test"), Array("ns1", "ns2", "ns3"), "tbl")
+    checkResolution("`db.tbl`", None, Array.empty, "db.tbl")
+    checkResolution("parquet.`file:/tmp/db.tbl`", None, Array("parquet"), "file:/tmp/db.tbl")
+    checkResolution("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", None,
+      Array("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")
+  }
+
+  test("resolve table identifier") {
+    checkTableResolution("tbl", Some(TableIdentifier("tbl")))
+    checkTableResolution("db.tbl", Some(TableIdentifier("tbl", Some("db"))))
+    checkTableResolution("prod.func", None)
+    checkTableResolution("ns1.ns2.tbl", None)
+    checkTableResolution("prod.db.tbl", None)
+    checkTableResolution("`db.tbl`", Some(TableIdentifier("db.tbl")))
+    checkTableResolution("parquet.`file:/tmp/db.tbl`",
+      Some(TableIdentifier("file:/tmp/db.tbl", Some("parquet"))))
+    checkTableResolution("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`",
+      Some(TableIdentifier("s3://buck/tmp/abc.json", Some("org.apache.spark.sql.json"))))
+  }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
index 9f33feb..8812684 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
@@ -234,6 +234,9 @@ case class MyParser(spark: SparkSession, delegate: ParserInterface) extends Pars
   override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier =
     delegate.parseFunctionIdentifier(sqlText)
 
+  override def parseMultipartIdentifier(sqlText: String): Seq[String] =
+    delegate.parseMultipartIdentifier(sqlText)
+
   override def parseTableSchema(sqlText: String): StructType =
     delegate.parseTableSchema(sqlText)
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org