You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Ryan Blue <rb...@netflix.com.INVALID> on 2019/02/04 01:51:20 UTC

Re: [DISCUSS] Identifiers with multi-catalog support

Xiao,

After reading the document you attached, I don’t think this problem affects
table resolution. The two resolution algorithms are orthogonal.

The 4th step in the algorithm for column resolution is this:

If there is more than one possible match, then the reference is ambiguous
and we are unable to resolve it. In this case, an AnalysisException is
thrown.

If a column reference is ambiguous, then the query fails analysis. There
can be just one match. That rule simplifies the problem quite a bit because
there isn’t some priority for choosing which one to use when a reference is
ambiguous.

The change that the table identifier work will introduce is a larger set of
names that identify tables. Instead of t1 or db1.t1, we will also have
c1.db1.t1. We may also have a situation where db1.t1 exists in two
catalogs, c1 and c2. If that’s the case, then most of the time there is no
problem because tables must be referenced to be added to the space used for
column resolution. In other words, you’d have to join c1.db1.t1 with
c2.db1.t1 to cause a possible conflict. If that query referenced a column
db1.t1.i1 and both tables had column i1, then the reference is ambiguous
and would fail analysis.

There are a few things to note about this:

   1. This problem is only “new” in the case of joining across catalogs,
   which isn’t currently possible: it won’t catch any existing queries.
   2. The same problem happens with ambiguous table references already:
   db1.t1 and db2.t1 can already introduce an ambiguous reference
   3. If there is a situation where multiple tables match or multiple
   columns match due to a conflict between levels, like db and catalog, then
   step #4 applies again and analysis fails because of an ambiguous reference.
   Likewise for columns.

rb

On Wed, Jan 23, 2019 at 9:58 PM Xiao Li <li...@databricks.com> wrote:

> This is the PR I mentioned in the meeting:
> https://github.com/apache/spark/pull/17185 This was merged into Spark
> 2.4. Now, the traditional DBMS users can easily migrate their previous
> queries to Spark SQL.
>
> *Attached doc* is the design doc by* Sunitha Kambhampati*, who is working
> for IBM BigInsight, STC and then CODAIT. She did a great work! She compared
> Spark with most mainstream DBMS and made Spark consistent with the others.
>
> In the doc, Table B shows the challenges we might face. Basically, the
> database name, catalog name, table name, and column name could be the same.
> Different resolution rules could cause different plans and then Spark will
> return results. Since this is very fundamental, we need to decide it in the
> early stage and also clearly document in the public Spark SQL doc. Our end
> users can follow it.
>
> To ensure we do not introduce the behavior changes, based on my
> understanding, we ONLY can add the new cases after the existing cases
> [which are mentioned in the doc], instead of introducing a brand-new
> resolution rule.
>
> Also, if we consider the nested columns, the cases become more complex.*
> DB Tsai* raised it offline with me multiple times about various issues in
> nested column handling. His company heavily used the nested columns. I
> think we should also consider nested fields in the resolution design;
> otherwise, we could easily break the existing behaviors. Spark SQL does not
> have enough test cases for nested columns, to be honest.
>
> Cheers,
>
> Xiao
>
> On Tue, Jan 22, 2019 at 12:24 PM Ryan Blue <rb...@netflix.com.invalid>
> wrote:
>
>> Thanks for reviewing this! I'll create an SPIP doc and issue for it and
>> call a vote.
>>
>> On Tue, Jan 22, 2019 at 11:41 AM Matt Cheah <mc...@palantir.com> wrote:
>>
>>> +1 for n-part namespace as proposed. Agree that a short SPIP would be
>>> appropriate for this. Perhaps also a JIRA ticket?
>>>
>>>
>>>
>>> -Matt Cheah
>>>
>>>
>>>
>>> *From: *Felix Cheung <fe...@hotmail.com>
>>> *Date: *Sunday, January 20, 2019 at 4:48 PM
>>> *To: *"rblue@netflix.com" <rb...@netflix.com>, Spark Dev List <
>>> dev@spark.apache.org>
>>> *Subject: *Re: [DISCUSS] Identifiers with multi-catalog support
>>>
>>>
>>>
>>> +1 I like Ryan last mail. Thank you for putting it clearly (should be a
>>> spec/SPIP!)
>>>
>>>
>>>
>>> I agree and understand the need for 3 part id. However I don’t think we
>>> should make assumption that it must be or can only be as long as 3 parts.
>>> Once the catalog is identified (ie. The first part), the catalog should be
>>> responsible for resolving the namespace or schema etc. Agree also path is
>>> good idea to add to support file-based variant. Should separator be
>>> optional (perhaps in *space) to keep this extensible (it might not always
>>> be ‘.’)
>>>
>>>
>>>
>>> Also this whole scheme will need to play nice with column identifier as
>>> well.
>>>
>>>
>>>
>>>
>>> ------------------------------
>>>
>>> *From:* Ryan Blue <rb...@netflix.com.invalid>
>>> *Sent:* Thursday, January 17, 2019 11:38 AM
>>> *To:* Spark Dev List
>>> *Subject:* Re: [DISCUSS] Identifiers with multi-catalog support
>>>
>>>
>>>
>>> Any discussion on how Spark should manage identifiers when multiple
>>> catalogs are supported?
>>>
>>>
>>>
>>> I know this is an area where a lot of people are interested in making
>>> progress, and it is a blocker for both multi-catalog support and CTAS in
>>> DSv2.
>>>
>>>
>>>
>>> On Sun, Jan 13, 2019 at 2:22 PM Ryan Blue <rb...@netflix.com> wrote:
>>>
>>> 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
>>>
>>>
>>>
>>>
>>> --
>>>
>>> Ryan Blue
>>>
>>> Software Engineer
>>>
>>> Netflix
>>>
>>
>>
>> --
>> Ryan Blue
>> Software Engineer
>> Netflix
>>
>
>
> --
> [image:
> https://databricks.com/sparkaisummit/north-america?utm_source=email&utm_medium=signature]
>
> ---------------------------------------------------------------------
> To unsubscribe e-mail: dev-unsubscribe@spark.apache.org



-- 
Ryan Blue
Software Engineer
Netflix