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 2022/11/15 05:29:49 UTC

[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5959: Spark: Iceberg bug 5935 fix where some methods of Spark3Util do not set current session in spark's threadlocal

aokolnychyi commented on code in PR #5959:
URL: https://github.com/apache/iceberg/pull/5959#discussion_r1022334125


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java:
##########
@@ -122,6 +125,31 @@ public void testLoadIcebergCatalog() throws Exception {
         "Should retrieve underlying catalog class", catalog instanceof CachingCatalog);
   }
 
+  // Bug 5935
+  @Test
+  public void testCurrentSessionSetAsActive() throws Exception {
+    stopMetastoreAndSpark();
+    startMetastoreAndSpark();
+    Assert.assertFalse(spark.conf().contains(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION().key()));
+    String testTableName = "default.testtable";
+    SparkSession hiveSupportSession = spark.newSession();
+    try {
+      hiveSupportSession.conf().set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION().key(),
+        SparkSessionCatalog.class.getName());
+      hiveSupportSession.conf().set("spark.sql.catalog.spark_catalog.type", "hive");
+
+      // create an iceberg table
+      String tableString = String.format("CREATE TABLE %s (id bigint, data string)" +
+          " USING iceberg PARTITIONED BY (data) TBLPROPERTIES('%s'='%s')", testTableName,
+        TableProperties.FORMAT_VERSION, "2");
+      hiveSupportSession.sql(tableString);
+      // Now load the created iceberg table
+      Spark3Util.loadIcebergTable(hiveSupportSession, testTableName);

Review Comment:
   
   
   If I understand correctly, the use case is as follows:
   - We create a new Spark session with its own independent session state.
     - Both `SQLConf` and `CatalogManager` are part of the session state, meaning our new session has its own.
   - We configure the session catalog in the new session.
   - We create a table using the new session, which works because it sets the new session as active.
   - We access `catalogManager().currentCatalog()` on the new session and get a wrong catalog because the catalog manager uses active SQL conf instead of the session SQL conf.
   
   
   
   One may argue the existing Spark behavior is questionable. At least because `CatalogManager` has a correct mapping internally when we invoke it. That being said, it is probably easier to just set the passed session as active.
   
   I'll take a look at the implementation tomorrow.
   



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