You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/04/16 15:41:01 UTC

[GitHub] [iceberg] RussellSpitzer opened a new issue #2488: Unexpected behavior with SparkSessionCatalog when Iceberg Metastore differs from Spark Metastore

RussellSpitzer opened a new issue #2488:
URL: https://github.com/apache/iceberg/issues/2488


   Our current configuration of the Spark3 Spark session catalog allows you to set the value of the metastore either by inheriting it from the Hadoop Configuration
   
   ```
   spark.hadoop.hive.metastore.uris=thrift://localhost:9083
   ```
   
   or by specifying it for the catalog itself
   
   ```
   spark.sql.catalog.spark_catalog.uri=thrift://localhost:9083 
   ```
   
   Or a user can use a non Hive based catalog for the Session or Iceberg table. The key issue here is that if these catalogs differ we can end up with a lot of weird situations.
   
   For example:
   Say we configure only "spark_catalog.uri", This will set the Iceberg metastore to a value but leave the Spark Session catalog on it's default value (in my local case derby). This means that almost all calls to database will be done on derby and invisible to Iceberg. So I can end up with weird behavior like
   
   ```scala
   scala> spark.sql("CREATE DATABASE catset")
   21/04/16 10:18:56 WARN ObjectStore: Failed to get database catset, returning NoSuchObjectException
   res4: org.apache.spark.sql.DataFrame = []
   scala> spark.sql("CREATE TABLE catset.foo (x int) USING iceberg")
   java.lang.RuntimeException: Metastore operation failed for catset.foo
   ```
   
   I have no problem making the database, but my CREATE command uses the Iceberg catalog which doesn't have the the database. So I get the "catset" not exists error. I
   
   
   
   I think to address this we need to disallow configuring the sparksession catalog with a different catalog type than the Iceberg catalog. This means we only actually allow for the sparksession catalog to be coupled with a hive metastore which is also configured for the delegate session catalog.


-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] itachi-sharingan commented on issue #2488: Forbid setting "uri" for SparkSessionCatalog when Hive is used.

Posted by GitBox <gi...@apache.org>.
itachi-sharingan commented on issue #2488:
URL: https://github.com/apache/iceberg/issues/2488#issuecomment-919412766


   There is a weird situation here, I think I know where to put the check but not sure I understand the problem correctly. Please bear with me:-
   Solution:
   Put a check in SparkSessionCatalog initialize function -> if option map contains key "type" as hive then there should not be a "uri" key in option.
   
   Problem(as far as I understand):-
   
   - if we configure two different thrift servers in Hadoop conf(i.e spark.hadoop.hive.metastore.uris) and iceberg cons(i.e spark.sql.catalog.spark_catalog.uri) then when try to create database the first one gets used and if try to create table then the second one gets used.
   
   Things I think I know:-
   
   - The uri configured in iceberg config(i.e spark.sql.catalog.spark_catalog.uri) always overrides the one in hadoop config(i.e spark.hadoop.hive.metastore.uris).
   - spark has a default catalog which acts as backup option(delegate catalog) if a particular table or database in spark sql ddl cannot be found in any new configured catalog metastore (given we have not configured spark_catalog itself).
   
   Things I am confused about:-
   - by configuring this - spark.sql.catalog.spark_catalog.uri=thrift://localhost:9083 and spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog, are we actually overriding the default catalog for spark. So now there is no backup catalog.
   - are the namespace functions listed in SparkSessionCatalog class for databases, I am assuming in database = namespace, is this wrong? If yes, what is the difference between the two?
   - why are "create database" and "create table" queries going to two different thrift servers if hadoop conf and iceberg conf are different or only spark_catalog.uri is configured?
   
   @rdblue @RussellSpitzer can you guys pls help?


-- 
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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2488: Forbid setting "uri" for SparkSessionCatalog when Hive is used.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2488:
URL: https://github.com/apache/iceberg/issues/2488#issuecomment-919470004


    1. The issue is that by configuring spark-catalog.uri=... to something different than the default session value isn't that it overrides the value. We instead end up configuring two different metastore connections. One connection to the Spark Delegate metastore and one connection to the Iceberg metastore. During our function calls we switch between these two catalogs which has a lot of unexpected behaviors. For example making a database in one catalog without making it in the other which leads to the error I explained above.
   
    2. Namespace is equivalent to "database" in this context. You can think of them as synonyms.
   
    3. Like I mentioned in one, the underlying issue is that the delegate catalog uses one metastore and the IcebergCatalog uses another. So rather than checking the same metastore for Iceberg and then non Iceberg tables, we first check one catalog for Iceberg and then check a different catalog for non-Iceberg. 
    So if I call "Create Database" this gets run on the Delegate Metastore, so now the delegate metastore has a new database. Now when I try to create an Iceberg table in that database, my requests are all routed to the Iceberg metastore where the database does not exist.


-- 
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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on issue #2488: Unexpected behavior with SparkSessionCatalog when Iceberg Metastore differs from Spark Metastore

Posted by GitBox <gi...@apache.org>.
flyrain commented on issue #2488:
URL: https://github.com/apache/iceberg/issues/2488#issuecomment-829863003


   +1 for the approach. Make sense to me if Spark build-in session catalog doesn't support Iceberg "Hadoop" type catalog.


-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] itachi-sharingan commented on issue #2488: Forbid setting "uri" for SparkSessionCatalog when Hive is used.

Posted by GitBox <gi...@apache.org>.
itachi-sharingan commented on issue #2488:
URL: https://github.com/apache/iceberg/issues/2488#issuecomment-920286924


   @RussellSpitzer thanks a lot for the clarification, can you please review the pr.


-- 
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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2488: Unexpected behavior with SparkSessionCatalog when Iceberg Metastore differs from Spark Metastore

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2488:
URL: https://github.com/apache/iceberg/issues/2488#issuecomment-918380919


   As recently discussed with @rdblue and @flyrain I think we have consensus that we don't want to allow having `hive.url` set when using the SparkSessionCatalog class since this is always incorrect. Gonna tag this with Good First Issue since it should be a relatively small task.


-- 
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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org