I think that the solution to this problem is to mix the two approaches by
supporting 3 identifier parts: catalog, namespace, and name, where
namespace can be an n-part identifier:

type Namespace = Seq[String]
case class CatalogIdentifier(space: Namespace, name: String)

This allows catalogs to work with the hierarchy of the external store, but
the catalog API only requires a few discovery methods to list namespaces
and to list each type of object in a namespace.

def listNamespaces(): Seq[Namespace]
def listNamespaces(space: Namespace, prefix: String): Seq[Namespace]
def listTables(space: Namespace): Seq[CatalogIdentifier]
def listViews(space: Namespace): Seq[CatalogIdentifier]
def listFunctions(space: Namespace): Seq[CatalogIdentifier]

The methods to list tables, views, or functions, would only return
identifiers for the type queried, not namespaces or the other objects.

The SQL parser would be updated so that identifiers are parsed to
UnresovledIdentifier(parts:
Seq[String]), and resolution would work like this pseudo-code:

def resolveIdentifier(ident: UnresolvedIdentifier): (CatalogPlugin,
CatalogIdentifier) = {
  val maybeCatalog = sparkSession.catalog(ident.parts.head)
  ident.parts match {
    case Seq(catalogName, *space, name) if catalog.isDefined =>
      (maybeCatalog.get, CatalogIdentifier(space, name))
    case Seq(*space, name) =>
      (sparkSession.defaultCatalog, CatalogIdentifier(space, name))
  }
}

I think this is a good approach because it allows Spark users to reference
or discovery any name in the hierarchy of an external store, it uses a few
well-defined methods for discovery, and makes name hierarchy a user concern.

   - SHOW (DATABASES|SCHEMAS|NAMESPACES) would return the result of
   listNamespaces()
   - SHOW NAMESPACES LIKE a.b% would return the result of
listNamespaces(Seq("a"),
   "b")
   - USE a.b would set the current namespace to Seq("a", "b")
   - SHOW TABLES would return the result of listTables(currentNamespace)

Also, I think that we could generalize this a little more to support
path-based tables by adding a path to CatalogIdentifier, either as a
namespace or as a separate optional string. Then, the identifier passed to
a catalog would work for either a path-based table or a catalog table,
without needing a path-based catalog API.

Thoughts?

On Sun, Jan 13, 2019 at 1:38 PM Ryan Blue <rb...@netflix.com> wrote:

> In the DSv2 sync up, we tried to discuss the Table metadata proposal but
> were side-tracked on its use of TableIdentifier. There were good points
> about how Spark should identify tables, views, functions, etc, and I want
> to start a discussion here.
>
> Identifiers are orthogonal to the TableCatalog proposal that can be
> updated to use whatever identifier class we choose. That proposal is
> concerned with what information should be passed to define a table, and how
> to pass that information.
>
> The main question for *this* discussion is: *how should Spark identify
> tables, views, and functions when it supports multiple catalogs?*
>
> There are two main approaches:
>
>    1. Use a 3-part identifier, catalog.database.table
>    2. Use an identifier with an arbitrary number of parts
>
> *Option 1: use 3-part identifiers*
>
> The argument for option #1 is that it is simple. If an external data store
> has additional logical hierarchy layers, then that hierarchy would be
> mapped to multiple catalogs in Spark. Spark can support show tables and
> show databases without much trouble. This is the approach used by Presto,
> so there is some precedent for it.
>
> The drawback is that mapping a more complex hierarchy into Spark requires
> more configuration. If an external DB has a 3-level hierarchy — say, for
> example, schema.database.table — then option #1 requires users to
> configure a catalog for each top-level structure, each schema. When a new
> schema is added, it is not automatically accessible.
>
> Catalog implementations could use session options could provide a rough
> work-around by changing the plugin’s “current” schema. I think this is an
> anti-pattern, so another strike against this option is that it encourages
> bad practices.
>
> *Option 2: use n-part identifiers*
>
> That drawback for option #1 is the main argument for option #2: Spark
> should allow users to easily interact with the native structure of an
> external store. For option #2, a full identifier would be an
> arbitrary-length list of identifiers. For the example above, using
> catalog.schema.database.table is allowed. An identifier would be
> something like this:
>
> case class CatalogIdentifier(parts: Seq[String])
>
> The problem with option #2 is how to implement a listing and discovery
> API, for operations like SHOW TABLES. If the catalog API requires a 
> list(ident:
> CatalogIdentifier), what does it return? There is no guarantee that the
> listed objects are tables and not nested namespaces. How would Spark handle
> arbitrary nesting that differs across catalogs?
>
> Hopefully, I’ve captured the design question well enough for a productive
> discussion. Thanks!
>
> rb
> --
> Ryan Blue
> Software Engineer
> Netflix
>


-- 
Ryan Blue
Software Engineer
Netflix

Reply via email to