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/07/14 07:07:49 UTC

[GitHub] [iceberg] szehon-ho commented on a change in pull request #2792: [SPARK] Allow spark catalogs to have hadoop configuration overrides p…

szehon-ho commented on a change in pull request #2792:
URL: https://github.com/apache/iceberg/pull/2792#discussion_r669336838



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.iceberg.spark.source;
+
+import java.util.Map;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.KryoHelpers;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.SparkCatalog;
+import org.apache.iceberg.spark.SparkCatalogTestBase;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+
+public class TestSparkCatalogHadoopOverrides extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        { "testhive", SparkCatalog.class.getName(),
+          ImmutableMap.of(
+            "type", "hive",
+            "default-namespace", "default",
+            "hadoop.fs.s3a.buffer.dir", "/tmp-overridden"
+        ) },
+        { "testhadoop", SparkCatalog.class.getName(),
+           ImmutableMap.of(
+            "type", "hadoop",
+            "hadoop.fs.s3a.buffer.dir", "/tmp-overridden"
+           ) },
+        { "spark_catalog", SparkSessionCatalog.class.getName(),
+          ImmutableMap.of(
+            "type", "hive",
+            "default-namespace", "default",
+            "hadoop.fs.s3a.buffer.dir", "/tmp-overridden"
+        ) }
+    };
+  }
+
+  public TestSparkCatalogHadoopOverrides(String catalogName,
+                                         String implementation,
+                                         Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Before
+  public void createTable() {
+    sql("CREATE TABLE IF NOT EXISTS %s (id bigint) USING iceberg", tableName(tableIdent.name()));
+  }
+
+  @After
+  public void dropTable() {
+    sql("DROP TABLE IF EXISTS %s", tableName(tableIdent.name()));
+  }
+
+  @Test
+  public void testTableFromCatalogHasOverrides() throws Exception {
+    Table table = getIcebergTableFromSparkCatalog();
+    Configuration conf = ((Configurable) table.io()).getConf();
+    String catalogOverrideFromTable = conf.get("fs.s3a.buffer.dir", "/whammies");
+    Assert.assertEquals(
+        "Iceberg tables from spark should have the overridden hadoop configurations from the spark config",
+        "/tmp-overridden", catalogOverrideFromTable);
+  }
+
+  @Test
+  public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception {
+    Table table = getIcebergTableFromSparkCatalog();
+    Configuration originalConf = ((Configurable) table.io()).getConf();
+    String catalogOverrideFromTable = originalConf.get("fs.s3a.buffer.dir", "/whammies");
+    Assert.assertEquals(

Review comment:
       Nit: is there any value to run the same test assert again as the previous test?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java
##########
@@ -32,12 +33,15 @@
 import org.apache.iceberg.transforms.Transform;
 import org.apache.iceberg.transforms.UnknownTransform;
 import org.apache.iceberg.util.Pair;
+import org.apache.spark.sql.SparkSession;
 import org.apache.spark.util.SerializableConfiguration;
 
 public class SparkUtil {
   private SparkUtil() {
   }
 
+  public static final String ICEBERG_CATALOG_PREFIX = "spark.sql.catalog";

Review comment:
       Nit: the name is maybe a bit misleading?  (spark_catalog_conf_prefix?)
   
   Also not sure, but looks like other constants like this are private static final, ie IcebergSource?

##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java
##########
@@ -99,4 +103,32 @@ public static void validatePartitionTransforms(PartitionSpec spec) {
       }
     }
   }
+
+  /**
+   * Pulls any Catalog specific overrides for the Hadoop conf from the current SparkSession, which can be
+   * set via spark.sql.catalog.$catalogName.hadoop.*
+   *
+   * The SparkCatalog allows for hadoop configurations to be overridden per catalog, by setting
+   * them on the SQLConf, where the following will add the property "fs.default.name" with value
+   * "hdfs://hanksnamenode:8020" to the catalog's hadoop configuration.
+   *   SparkSession.builder()
+   *     .config(s"spark.sql.catalog.$catalogName.hadoop.fs.default.name", "hdfs://hanksnamenode:8020")
+   *     .getOrCreate()
+   * @param spark The current Spark session
+   * @param catalogName Name of the catalog to find overrides for.
+   * @return the Hadoop Configuration that should be used for this catalog, with catalog specific overrides applied.
+   */
+  public static Configuration hadoopConfCatalogOverrides(SparkSession spark, String catalogName) {
+    // Find keys for the catalog intended to be hadoop configurations
+    final String hadoopConfCatalogPrefix = String.format("%s.%s.%s", ICEBERG_CATALOG_PREFIX, catalogName, "hadoop.");
+    Configuration conf = spark.sessionState().newHadoopConf();
+    spark.sqlContext().conf().settings().forEach((k, v) -> {
+      // These checks are copied from `spark.sessionState().newHadoopConfWithOptions()`, which we

Review comment:
       I see we lost the synchronize { } call that we would have get from spark.sessionState.newHadoopConf (via getAllConfs method), should we use it as the SynchronizedMap expectes everyone synchronize when getting the map?  (Esp as this is a public API)




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