|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package org.apache.iceberg.spark.source; |
| 21 | + |
| 22 | +import java.util.Map; |
| 23 | +import org.apache.hadoop.conf.Configurable; |
| 24 | +import org.apache.hadoop.conf.Configuration; |
| 25 | +import org.apache.iceberg.KryoHelpers; |
| 26 | +import org.apache.iceberg.SerializableTable; |
| 27 | +import org.apache.iceberg.Table; |
| 28 | +import org.apache.iceberg.TestHelpers; |
| 29 | +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; |
| 30 | +import org.apache.iceberg.spark.SparkCatalog; |
| 31 | +import org.apache.iceberg.spark.SparkCatalogTestBase; |
| 32 | +import org.apache.iceberg.spark.SparkSessionCatalog; |
| 33 | +import org.apache.spark.sql.connector.catalog.Identifier; |
| 34 | +import org.apache.spark.sql.connector.catalog.TableCatalog; |
| 35 | +import org.junit.After; |
| 36 | +import org.junit.Assert; |
| 37 | +import org.junit.Before; |
| 38 | +import org.junit.Test; |
| 39 | +import org.junit.runners.Parameterized; |
| 40 | + |
| 41 | + |
| 42 | +public class TestSparkCatalogHadoopOverrides extends SparkCatalogTestBase { |
| 43 | + |
| 44 | + private static final String configToOverride = "fs.s3a.buffer.dir"; |
| 45 | + // prepend "hadoop." so that the test base formats SQLConf correctly |
| 46 | + // as `spark.sql.catalogs.<catalogName>.hadoop.<configToOverride> |
| 47 | + private static final String hadoopPrefixedConfigToOverride = "hadoop." + configToOverride; |
| 48 | + private static final String configOverrideValue = "/tmp-overridden"; |
| 49 | + |
| 50 | + @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") |
| 51 | + public static Object[][] parameters() { |
| 52 | + return new Object[][] { |
| 53 | + { "testhive", SparkCatalog.class.getName(), |
| 54 | + ImmutableMap.of( |
| 55 | + "type", "hive", |
| 56 | + "default-namespace", "default", |
| 57 | + hadoopPrefixedConfigToOverride, configOverrideValue |
| 58 | + ) }, |
| 59 | + { "testhadoop", SparkCatalog.class.getName(), |
| 60 | + ImmutableMap.of( |
| 61 | + "type", "hadoop", |
| 62 | + hadoopPrefixedConfigToOverride, configOverrideValue |
| 63 | + ) }, |
| 64 | + { "spark_catalog", SparkSessionCatalog.class.getName(), |
| 65 | + ImmutableMap.of( |
| 66 | + "type", "hive", |
| 67 | + "default-namespace", "default", |
| 68 | + hadoopPrefixedConfigToOverride, configOverrideValue |
| 69 | + ) } |
| 70 | + }; |
| 71 | + } |
| 72 | + |
| 73 | + public TestSparkCatalogHadoopOverrides(String catalogName, |
| 74 | + String implementation, |
| 75 | + Map<String, String> config) { |
| 76 | + super(catalogName, implementation, config); |
| 77 | + } |
| 78 | + |
| 79 | + @Before |
| 80 | + public void createTable() { |
| 81 | + sql("CREATE TABLE IF NOT EXISTS %s (id bigint) USING iceberg", tableName(tableIdent.name())); |
| 82 | + } |
| 83 | + |
| 84 | + @After |
| 85 | + public void dropTable() { |
| 86 | + sql("DROP TABLE IF EXISTS %s", tableName(tableIdent.name())); |
| 87 | + } |
| 88 | + |
| 89 | + @Test |
| 90 | + public void testTableFromCatalogHasOverrides() throws Exception { |
| 91 | + Table table = getIcebergTableFromSparkCatalog(); |
| 92 | + Configuration conf = ((Configurable) table.io()).getConf(); |
| 93 | + String actualCatalogOverride = conf.get(configToOverride, "/whammies"); |
| 94 | + Assert.assertEquals( |
| 95 | + "Iceberg tables from spark should have the overridden hadoop configurations from the spark config", |
| 96 | + configOverrideValue, actualCatalogOverride); |
| 97 | + } |
| 98 | + |
| 99 | + @Test |
| 100 | + public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception { |
| 101 | + Table table = getIcebergTableFromSparkCatalog(); |
| 102 | + Configuration originalConf = ((Configurable) table.io()).getConf(); |
| 103 | + String actualCatalogOverride = originalConf.get(configToOverride, "/whammies"); |
| 104 | + Assert.assertEquals( |
| 105 | + "Iceberg tables from spark should have the overridden hadoop configurations from the spark config", |
| 106 | + configOverrideValue, actualCatalogOverride); |
| 107 | + |
| 108 | + // Now convert to SerializableTable and ensure overridden property is still present. |
| 109 | + Table serializableTable = SerializableTable.copyOf(table); |
| 110 | + Table kryoSerializedTable = KryoHelpers.roundTripSerialize(SerializableTable.copyOf(table)); |
| 111 | + Configuration configFromKryoSerde = ((Configurable) kryoSerializedTable.io()).getConf(); |
| 112 | + String kryoSerializedCatalogOverride = configFromKryoSerde.get(configToOverride, "/whammies"); |
| 113 | + Assert.assertEquals( |
| 114 | + "Tables serialized with Kryo serialization should retain overridden hadoop configuration properties", |
| 115 | + configOverrideValue, kryoSerializedCatalogOverride); |
| 116 | + |
| 117 | + // Do the same for Java based serde |
| 118 | + Table javaSerializedTable = TestHelpers.roundTripSerialize(serializableTable); |
| 119 | + Configuration configFromJavaSerde = ((Configurable) javaSerializedTable.io()).getConf(); |
| 120 | + String javaSerializedCatalogOverride = configFromJavaSerde.get(configToOverride, "/whammies"); |
| 121 | + Assert.assertEquals( |
| 122 | + "Tables serialized with Java serialization should retain overridden hadoop configuration properties", |
| 123 | + configOverrideValue, javaSerializedCatalogOverride); |
| 124 | + } |
| 125 | + |
| 126 | + @SuppressWarnings("ThrowSpecificity") |
| 127 | + private Table getIcebergTableFromSparkCatalog() throws Exception { |
| 128 | + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); |
| 129 | + TableCatalog catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); |
| 130 | + SparkTable sparkTable = (SparkTable) catalog.loadTable(identifier); |
| 131 | + return sparkTable.table(); |
| 132 | + } |
| 133 | +} |
0 commit comments