This is an automated email from the ASF dual-hosted git repository.
ayushsaxena pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new 311692d311a HIVE-27013: Iceberg: Provide an option to enable iceberg
manifest caching for all catalogs. (#4088). (Ayush Saxena, reviewed by Denys
Kuzmenko)
311692d311a is described below
commit 311692d311a70632c0d688342051d5a2cfbf08ce
Author: Ayush Saxena <[email protected]>
AuthorDate: Fri Mar 10 12:56:47 2023 +0530
HIVE-27013: Iceberg: Provide an option to enable iceberg manifest caching
for all catalogs. (#4088). (Ayush Saxena, reviewed by Denys Kuzmenko)
---
.../java/org/apache/iceberg/hive/HiveCatalog.java | 10 +++++++---
.../main/java/org/apache/iceberg/mr/Catalogs.java | 19 +++++++++++++------
.../org/apache/iceberg/mr/InputFormatConfig.java | 1 +
.../java/org/apache/iceberg/mr/TestCatalogs.java | 21 +++++++++++++++++++++
4 files changed, 42 insertions(+), 9 deletions(-)
diff --git
a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
index 80f7d4065bb..6cccb0c01a2 100644
---
a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
+++
b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
@@ -98,8 +98,12 @@ public class HiveCatalog extends BaseMetastoreCatalog
implements SupportsNamespa
this.listAllTables =
Boolean.parseBoolean(properties.getOrDefault(LIST_ALL_TABLES,
LIST_ALL_TABLES_DEFAULT));
String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
- this.fileIO = fileIOImpl == null ? new HadoopFileIO(conf) :
CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
-
+ if (fileIOImpl == null) {
+ this.fileIO = new HadoopFileIO(conf);
+ this.fileIO.initialize(properties);
+ } else {
+ this.fileIO = CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+ }
this.clients = new CachedClientPool(conf, properties);
}
@@ -535,7 +539,7 @@ public class HiveCatalog extends BaseMetastoreCatalog
implements SupportsNamespa
}
@Override
- protected Map<String, String> properties() {
+ public Map<String, String> properties() {
return catalogProperties == null ? ImmutableMap.of() : catalogProperties;
}
diff --git
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/Catalogs.java
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/Catalogs.java
index d77ede333d1..c58e7d07fc5 100644
--- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/Catalogs.java
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/Catalogs.java
@@ -23,7 +23,6 @@ import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.Set;
-import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.CatalogUtil;
@@ -38,7 +37,6 @@ import org.apache.iceberg.hadoop.HadoopTables;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.relocated.com.google.common.collect.Streams;
/**
* Class for catalog resolution and accessing the common functions for {@link
Catalog} API.
@@ -243,10 +241,19 @@ public final class Catalogs {
* @return complete map of catalog properties
*/
private static Map<String, String> getCatalogProperties(Configuration conf,
String catalogName, String catalogType) {
- String keyPrefix = InputFormatConfig.CATALOG_CONFIG_PREFIX + catalogName;
- Map<String, String> catalogProperties = Streams.stream(conf.iterator())
- .filter(e -> e.getKey().startsWith(keyPrefix))
- .collect(Collectors.toMap(e ->
e.getKey().substring(keyPrefix.length() + 1), Map.Entry::getValue));
+ Map<String, String> catalogProperties = Maps.newHashMap();
+ conf.forEach(config -> {
+ if
(config.getKey().startsWith(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX)) {
+ catalogProperties.putIfAbsent(
+
config.getKey().substring(InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX.length()),
+ config.getValue());
+ } else if
(config.getKey().startsWith(InputFormatConfig.CATALOG_CONFIG_PREFIX +
catalogName)) {
+ catalogProperties.put(
+ config.getKey().substring((InputFormatConfig.CATALOG_CONFIG_PREFIX
+ catalogName).length() + 1),
+ config.getValue());
+ }
+ });
+
return addCatalogPropertiesIfMissing(conf, catalogType, catalogProperties);
}
diff --git
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java
index b7a02ed3653..2b64c79160c 100644
---
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java
+++
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java
@@ -106,6 +106,7 @@ public class InputFormatConfig {
public static final String CATALOG_TYPE_TEMPLATE = "iceberg.catalog.%s.type";
public static final String CATALOG_WAREHOUSE_TEMPLATE =
"iceberg.catalog.%s.warehouse";
public static final String CATALOG_CLASS_TEMPLATE =
"iceberg.catalog.%s.catalog-impl";
+ public static final String CATALOG_DEFAULT_CONFIG_PREFIX =
"iceberg.catalog-default.";
public enum InMemoryDataModel {
PIG,
diff --git
a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java
b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java
index 9b3ee40d1a3..e9f0bd24de4 100644
---
a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java
+++
b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/TestCatalogs.java
@@ -338,6 +338,27 @@ public class TestCatalogs {
"Unknown catalog type:", () -> Catalogs.loadCatalog(conf,
catalogName));
}
+ @Test
+ public void testDefaultCatalogProperties() {
+ String catalogProperty = "io.manifest.cache-enabled";
+ // Set global property
+ final String defaultCatalogProperty =
InputFormatConfig.CATALOG_DEFAULT_CONFIG_PREFIX + catalogProperty;
+ conf.setBoolean(defaultCatalogProperty, true);
+ HiveCatalog defaultCatalog = (HiveCatalog) Catalogs.loadCatalog(conf,
null).get();
+ Assert.assertEquals("true",
defaultCatalog.properties().get(catalogProperty));
+ Assert.assertEquals("true",
+ defaultCatalog.newTableOps(TableIdentifier.of("default",
"iceberg")).io().properties().get(catalogProperty));
+
+ // set property at catalog level, and that should take precedence over the
global property.
+ conf.setBoolean(
+ String.format("%s%s.%s", InputFormatConfig.CATALOG_CONFIG_PREFIX,
Catalogs.ICEBERG_DEFAULT_CATALOG_NAME,
+ catalogProperty), false);
+ defaultCatalog = (HiveCatalog) Catalogs.loadCatalog(conf, null).get();
+ Assert.assertEquals("false",
defaultCatalog.properties().get(catalogProperty));
+ Assert.assertEquals("false",
+ defaultCatalog.newTableOps(TableIdentifier.of("default",
"iceberg")).io().properties().get(catalogProperty));
+ }
+
public static class CustomHadoopCatalog extends HadoopCatalog {
public CustomHadoopCatalog() {