You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2022/05/17 22:51:30 UTC

[iceberg] branch master updated: Core: Allow controlling table properties through catalog config (#4011)

This is an automated email from the ASF dual-hosted git repository.

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 06fcdbb42 Core: Allow controlling table properties through catalog config (#4011)
06fcdbb42 is described below

commit 06fcdbb42d596d5237ad148a5c9a36006148ee7b
Author: Ashish Singh <as...@pinterest.com>
AuthorDate: Tue May 17 15:51:25 2022 -0700

    Core: Allow controlling table properties through catalog config (#4011)
    
    Co-authored-by: Rajarshi Sarkar <sr...@amazon.com>
---
 .../iceberg/aws/glue/TestGlueCatalogTable.java     | 51 ++++++++++++++++++++++
 .../org/apache/iceberg/aws/glue/GlueCatalog.java   | 16 ++++++-
 .../apache/iceberg/aws/glue/TestGlueCatalog.java   | 24 ++++++++++
 .../org/apache/iceberg/BaseMetastoreCatalog.java   | 46 ++++++++++++++-----
 .../java/org/apache/iceberg/CatalogProperties.java |  2 +
 .../org/apache/iceberg/hadoop/HadoopCatalog.java   |  8 ++++
 .../java/org/apache/iceberg/util/PropertyUtil.java |  2 +-
 .../apache/iceberg/hadoop/HadoopTableTestBase.java | 14 +++++-
 .../apache/iceberg/hadoop/TestHadoopCatalog.java   | 42 ++++++++++++++++++
 .../java/org/apache/iceberg/hive/HiveCatalog.java  |  8 ++++
 .../org/apache/iceberg/hive/TestHiveCatalog.java   | 51 ++++++++++++++++++++++
 11 files changed, 250 insertions(+), 14 deletions(-)

diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java
index 8429e0f69..f493ccd73 100644
--- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java
+++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java
@@ -41,6 +41,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
 import org.junit.Assert;
 import org.junit.Test;
 import software.amazon.awssdk.services.glue.model.Column;
@@ -368,4 +369,54 @@ public class TestGlueCatalogTable extends GlueTestBase {
     );
     Assert.assertEquals("Columns do not match", expectedColumns, actualColumns);
   }
+
+  @Test
+  public void testTablePropsDefinedAtCatalogLevel() {
+    String namespace = createNamespace();
+    String tableName = getRandomName();
+    TableIdentifier tableIdent = TableIdentifier.of(namespace, tableName);
+    ImmutableMap<String, String> catalogProps = ImmutableMap.of(
+        "table-default.key1", "catalog-default-key1",
+        "table-default.key2", "catalog-default-key2",
+        "table-default.key3", "catalog-default-key3",
+        "table-override.key3", "catalog-override-key3",
+        "table-override.key4", "catalog-override-key4",
+        "warehouse", "s3://" + testBucketName + "/" + testPathPrefix);
+
+    glueCatalog.initialize("glue", catalogProps);
+
+    Schema schema = new Schema(
+        NestedField.required(3, "id", Types.IntegerType.get(), "unique ID"),
+        NestedField.required(4, "data", Types.StringType.get())
+    );
+
+    org.apache.iceberg.Table table = glueCatalog.buildTable(tableIdent, schema)
+        .withProperty("key2", "table-key2")
+        .withProperty("key3", "table-key3")
+        .withProperty("key5", "table-key5")
+        .create();
+
+    Assert.assertEquals(
+        "Table defaults set for the catalog must be added to the table properties.",
+        "catalog-default-key1",
+        table.properties().get("key1"));
+    Assert.assertEquals(
+        "Table property must override table default properties set at catalog level.",
+        "table-key2",
+        table.properties().get("key2"));
+    Assert.assertEquals(
+        "Table property override set at catalog level must override table default" +
+            " properties set at catalog level and table property specified.",
+        "catalog-override-key3",
+        table.properties().get("key3"));
+    Assert.assertEquals(
+        "Table override not in table props or defaults should be added to table properties",
+        "catalog-override-key4",
+        table.properties().get("key4"));
+    Assert.assertEquals(
+        "Table properties without any catalog level default or override should be added to table" +
+            " properties.",
+        "table-key5",
+        table.properties().get("key5"));
+  }
 }
diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
index 59752e350..3c04a1bfb 100644
--- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
+++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
@@ -21,6 +21,7 @@ package org.apache.iceberg.aws.glue;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -92,7 +93,7 @@ public class GlueCatalog extends BaseMetastoreCatalog
   private FileIO fileIO;
   private LockManager lockManager;
   private CloseableGroup closeableGroup;
-  private Map<String, String> catalogProperties;
+  private Map<String, String> catalogProperties = Collections.emptyMap();
 
   // Attempt to set versionId if available on the path
   private static final DynMethods.UnboundMethod SET_VERSION_ID = DynMethods.builder("versionId")
@@ -110,6 +111,7 @@ public class GlueCatalog extends BaseMetastoreCatalog
 
   @Override
   public void initialize(String name, Map<String, String> properties) {
+    this.catalogProperties = properties;
     AwsClientFactory awsClientFactory;
     FileIO catalogFileIO;
     if (PropertyUtil.propertyAsBoolean(
@@ -162,6 +164,13 @@ public class GlueCatalog extends BaseMetastoreCatalog
     }
   }
 
+  @VisibleForTesting
+  void initialize(String name, String path, AwsProperties properties, GlueClient client,
+      LockManager lock, FileIO io, Map<String, String> catalogProps) {
+    this.catalogProperties = catalogProps;
+    initialize(name, path, properties, client, lock, io);
+  }
+
   @VisibleForTesting
   void initialize(String name, String path, AwsProperties properties, GlueClient client, LockManager lock, FileIO io) {
     Preconditions.checkArgument(path != null && path.length() > 0,
@@ -495,4 +504,9 @@ public class GlueCatalog extends BaseMetastoreCatalog
   public void setConf(Configuration conf) {
     this.hadoopConf = conf;
   }
+
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties;
+  }
 }
diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java
index b5db60662..b81d1cf0c 100644
--- a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java
+++ b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java
@@ -455,4 +455,28 @@ public class TestGlueCatalog {
         .when(glue).updateDatabase(Mockito.any(UpdateDatabaseRequest.class));
     glueCatalog.removeProperties(Namespace.of("db1"), Sets.newHashSet("key"));
   }
+
+  @Test
+  public void testTablePropsDefinedAtCatalogLevel() {
+    ImmutableMap<String, String> catalogProps = ImmutableMap.of(
+        "table-default.key1", "catalog-default-key1",
+        "table-default.key2", "catalog-default-key2",
+        "table-default.key3", "catalog-default-key3",
+        "table-override.key3", "catalog-override-key3",
+        "table-override.key4", "catalog-override-key4");
+    glueCatalog.initialize(CATALOG_NAME, WAREHOUSE_PATH, new AwsProperties(), glue,
+        LockManagers.defaultLockManager(), null, catalogProps);
+    Map<String, String> properties = glueCatalog.properties();
+    Assert.assertFalse(properties.isEmpty());
+    Assert.assertTrue(properties.containsKey("table-default.key1"));
+    Assert.assertEquals("catalog-default-key1", properties.get("table-default.key1"));
+    Assert.assertTrue(properties.containsKey("table-default.key2"));
+    Assert.assertEquals("catalog-default-key2", properties.get("table-default.key2"));
+    Assert.assertTrue(properties.containsKey("table-default.key3"));
+    Assert.assertEquals("catalog-default-key3", properties.get("table-default.key3"));
+    Assert.assertTrue(properties.containsKey("table-override.key3"));
+    Assert.assertEquals("catalog-override-key3", properties.get("table-override.key3"));
+    Assert.assertTrue(properties.containsKey("table-override.key4"));
+    Assert.assertEquals("catalog-override-key4", properties.get("table-override.key4"));
+  }
 }
diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
index 8fad3839f..e4ad6a7a2 100644
--- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
+++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
@@ -19,6 +19,7 @@
 
 package org.apache.iceberg;
 
+import java.util.Collections;
 import java.util.Map;
 import org.apache.iceberg.catalog.Catalog;
 import org.apache.iceberg.catalog.TableIdentifier;
@@ -27,7 +28,8 @@ import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
-import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.PropertyUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -94,6 +96,10 @@ public abstract class BaseMetastoreCatalog implements Catalog {
     return true;
   }
 
+  protected Map<String, String> properties() {
+    return Collections.emptyMap();
+  }
+
   @Override
   public String toString() {
     return MoreObjects.toStringHelper(this).toString();
@@ -106,7 +112,7 @@ public abstract class BaseMetastoreCatalog implements Catalog {
   protected class BaseMetastoreCatalogTableBuilder implements TableBuilder {
     private final TableIdentifier identifier;
     private final Schema schema;
-    private final ImmutableMap.Builder<String, String> propertiesBuilder = ImmutableMap.builder();
+    private final Map<String, String> tableProperties = Maps.newHashMap();
     private PartitionSpec spec = PartitionSpec.unpartitioned();
     private SortOrder sortOrder = SortOrder.unsorted();
     private String location = null;
@@ -116,6 +122,7 @@ public abstract class BaseMetastoreCatalog implements Catalog {
 
       this.identifier = identifier;
       this.schema = schema;
+      this.tableProperties.putAll(tableDefaultProperties());
     }
 
     @Override
@@ -139,14 +146,14 @@ public abstract class BaseMetastoreCatalog implements Catalog {
     @Override
     public TableBuilder withProperties(Map<String, String> properties) {
       if (properties != null) {
-        propertiesBuilder.putAll(properties);
+        tableProperties.putAll(properties);
       }
       return this;
     }
 
     @Override
     public TableBuilder withProperty(String key, String value) {
-      propertiesBuilder.put(key, value);
+      tableProperties.put(key, value);
       return this;
     }
 
@@ -158,8 +165,8 @@ public abstract class BaseMetastoreCatalog implements Catalog {
       }
 
       String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
-      Map<String, String> properties = propertiesBuilder.build();
-      TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, sortOrder, baseLocation, properties);
+      tableProperties.putAll(tableOverrideProperties());
+      TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, sortOrder, baseLocation, tableProperties);
 
       try {
         ops.commit(null, metadata);
@@ -178,8 +185,8 @@ public abstract class BaseMetastoreCatalog implements Catalog {
       }
 
       String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
-      Map<String, String> properties = propertiesBuilder.build();
-      TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, sortOrder, baseLocation, properties);
+      tableProperties.putAll(tableOverrideProperties());
+      TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, sortOrder, baseLocation, tableProperties);
       return Transactions.createTableTransaction(identifier.toString(), ops, metadata);
     }
 
@@ -200,12 +207,13 @@ public abstract class BaseMetastoreCatalog implements Catalog {
       }
 
       TableMetadata metadata;
+      tableProperties.putAll(tableOverrideProperties());
       if (ops.current() != null) {
         String baseLocation = location != null ? location : ops.current().location();
-        metadata = ops.current().buildReplacement(schema, spec, sortOrder, baseLocation, propertiesBuilder.build());
+        metadata = ops.current().buildReplacement(schema, spec, sortOrder, baseLocation, tableProperties);
       } else {
         String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
-        metadata = TableMetadata.newTableMetadata(schema, spec, sortOrder, baseLocation, propertiesBuilder.build());
+        metadata = TableMetadata.newTableMetadata(schema, spec, sortOrder, baseLocation, tableProperties);
       }
 
       if (orCreate) {
@@ -214,6 +222,24 @@ public abstract class BaseMetastoreCatalog implements Catalog {
         return Transactions.replaceTableTransaction(identifier.toString(), ops, metadata);
       }
     }
+
+    /**
+     * Get default table properties set at Catalog level through catalog properties.
+     *
+     * @return default table properties specified in catalog properties
+     */
+    private Map<String, String> tableDefaultProperties() {
+      return PropertyUtil.propertiesWithPrefix(properties(), CatalogProperties.TABLE_DEFAULT_PREFIX);
+    }
+
+    /**
+     * Get table properties that are enforced at Catalog level through catalog properties.
+     *
+     * @return default table properties enforced through catalog properties
+     */
+    private Map<String, String> tableOverrideProperties() {
+      return PropertyUtil.propertiesWithPrefix(properties(), CatalogProperties.TABLE_OVERRIDE_PREFIX);
+    }
   }
 
   protected static String fullTableName(String catalogName, TableIdentifier identifier) {
diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java
index d5daedee8..447970cf6 100644
--- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java
+++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java
@@ -29,6 +29,8 @@ public class CatalogProperties {
   public static final String CATALOG_IMPL = "catalog-impl";
   public static final String FILE_IO_IMPL = "io-impl";
   public static final String WAREHOUSE_LOCATION = "warehouse";
+  public static final String TABLE_DEFAULT_PREFIX = "table-default.";
+  public static final String TABLE_OVERRIDE_PREFIX = "table-override.";
 
   /**
    * Controls whether the catalog will cache table entries upon load.
diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java
index eb4dd874f..b574b5c78 100644
--- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java
+++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.nio.file.AccessDeniedException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -93,12 +94,14 @@ public class HadoopCatalog extends BaseMetastoreCatalog implements Closeable, Su
   private FileIO fileIO;
   private LockManager lockManager;
   private boolean suppressPermissionError = false;
+  private Map<String, String> catalogProperties = Collections.emptyMap();
 
   public HadoopCatalog() {
   }
 
   @Override
   public void initialize(String name, Map<String, String> properties) {
+    this.catalogProperties = properties;
     String inputWarehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
     Preconditions.checkArgument(inputWarehouseLocation != null && inputWarehouseLocation.length() > 0,
         "Cannot initialize HadoopCatalog because warehousePath must not be null or empty");
@@ -393,6 +396,11 @@ public class HadoopCatalog extends BaseMetastoreCatalog implements Closeable, Su
     return conf;
   }
 
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties;
+  }
+
   private class HadoopCatalogTableBuilder extends BaseMetastoreCatalogTableBuilder {
     private final String defaultLocation;
 
diff --git a/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java b/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java
index 601f88fc0..1028bc5e0 100644
--- a/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java
+++ b/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java
@@ -88,7 +88,7 @@ public class PropertyUtil {
       return ImmutableMap.of();
     }
 
-    Preconditions.checkArgument(prefix != null, "prefix can't be null.");
+    Preconditions.checkArgument(prefix != null, "Invalid prefix: null");
 
     return properties.entrySet().stream()
         .filter(e -> e.getKey().startsWith(prefix))
diff --git a/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java
index 29643ab58..4cf0033bc 100644
--- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java
+++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopTableTestBase.java
@@ -24,7 +24,9 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.zip.GZIPOutputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.iceberg.CatalogProperties;
@@ -179,10 +181,18 @@ public class HadoopTableTestBase {
   }
 
   protected HadoopCatalog hadoopCatalog() throws IOException {
+    return hadoopCatalog(Collections.emptyMap());
+  }
+
+  protected HadoopCatalog hadoopCatalog(Map<String, String> catalogProperties) throws IOException {
     HadoopCatalog hadoopCatalog = new HadoopCatalog();
     hadoopCatalog.setConf(new Configuration());
-    hadoopCatalog.initialize("hadoop",
-            ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, temp.newFolder().getAbsolutePath()));
+    hadoopCatalog.initialize(
+        "hadoop",
+        ImmutableMap.<String, String>builder()
+            .putAll(catalogProperties)
+            .put(CatalogProperties.WAREHOUSE_LOCATION, temp.newFolder().getAbsolutePath())
+            .build());
     return hadoopCatalog;
   }
 }
diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java
index 244f01aaa..889cf2b2b 100644
--- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java
+++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java
@@ -547,4 +547,46 @@ public class TestHadoopCatalog extends HadoopTableTestBase {
     table.newAppend().appendFile(dataFile1).commit();
     table.newAppend().appendFile(dataFile2).commit();
   }
+
+  @Test
+  public void testTablePropsDefinedAtCatalogLevel() throws IOException {
+    TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl");
+    ImmutableMap<String, String> catalogProps = ImmutableMap.of(
+        "table-default.key1", "catalog-default-key1",
+        "table-default.key2", "catalog-default-key2",
+        "table-default.key3", "catalog-default-key3",
+        "table-override.key3", "catalog-override-key3",
+        "table-override.key4", "catalog-override-key4");
+
+    Table table = hadoopCatalog(catalogProps).buildTable(tableIdent, SCHEMA)
+        .withPartitionSpec(SPEC)
+        .withProperties(null)
+        .withProperty("key2", "table-key2")
+        .withProperty("key3", "table-key3")
+        .withProperty("key5", "table-key5")
+        .create();
+
+    Assert.assertEquals(
+        "Table defaults set for the catalog must be added to the table properties.",
+        "catalog-default-key1",
+        table.properties().get("key1"));
+    Assert.assertEquals(
+        "Table property must override table default properties set at catalog level.",
+        "table-key2",
+        table.properties().get("key2"));
+    Assert.assertEquals(
+        "Table property override set at catalog level must override table default" +
+            " properties set at catalog level and table property specified.",
+        "catalog-override-key3",
+        table.properties().get("key3"));
+    Assert.assertEquals(
+        "Table override not in table props or defaults should be added to table properties",
+        "catalog-override-key4",
+        table.properties().get("key4"));
+    Assert.assertEquals(
+        "Table properties without any catalog level default or override should be added to table" +
+            " properties.",
+        "table-key5",
+        table.properties().get("key5"));
+  }
 }
diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
index a3e96583f..7d1f84d91 100644
--- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
+++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
@@ -19,6 +19,7 @@
 
 package org.apache.iceberg.hive;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -73,12 +74,14 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
   private FileIO fileIO;
   private ClientPool<IMetaStoreClient, TException> clients;
   private boolean listAllTables = false;
+  private Map<String, String> catalogProperties = Collections.emptyMap();
 
   public HiveCatalog() {
   }
 
   @Override
   public void initialize(String inputName, Map<String, String> properties) {
+    this.catalogProperties = properties;
     this.name = inputName;
     if (conf == null) {
       LOG.warn("No Hadoop Configuration was set, using the default environment Configuration");
@@ -539,6 +542,11 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
     return conf;
   }
 
+  @Override
+  protected Map<String, String> properties() {
+    return catalogProperties;
+  }
+
   @VisibleForTesting
   void setListAllTables(boolean listAllTables) {
     this.listAllTables = listAllTables;
diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
index 69fa23f39..46ccb571a 100644
--- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
+++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.iceberg.AssertHelpers;
 import org.apache.iceberg.CachingCatalog;
 import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
 import org.apache.iceberg.DataFile;
 import org.apache.iceberg.DataFiles;
 import org.apache.iceberg.FileFormat;
@@ -659,4 +660,54 @@ public class TestHiveCatalog extends HiveMetastoreTest {
     Assert.assertEquals("Should have trailing slash stripped", wareHousePath, catalogWithSlash.getConf().get(
         HiveConf.ConfVars.METASTOREWAREHOUSE.varname));
   }
+
+  @Test
+  public void testTablePropsDefinedAtCatalogLevel() {
+    Schema schema = new Schema(
+        required(1, "id", Types.IntegerType.get(), "unique ID")
+    );
+    TableIdentifier tableIdent = TableIdentifier.of(DB_NAME, "tbl");
+
+    ImmutableMap<String, String> catalogProps = ImmutableMap.of(
+        "table-default.key1", "catalog-default-key1",
+        "table-default.key2", "catalog-default-key2",
+        "table-default.key3", "catalog-default-key3",
+        "table-override.key3", "catalog-override-key3",
+        "table-override.key4", "catalog-override-key4");
+    Catalog hiveCatalog = CatalogUtil.loadCatalog(HiveCatalog.class.getName(),
+        CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE, catalogProps, hiveConf);
+
+    try {
+      Table table = hiveCatalog.buildTable(tableIdent, schema)
+          .withProperty("key2", "table-key2")
+          .withProperty("key3", "table-key3")
+          .withProperty("key5", "table-key5")
+          .create();
+
+      Assert.assertEquals(
+          "Table defaults set for the catalog must be added to the table properties.",
+          "catalog-default-key1",
+          table.properties().get("key1"));
+      Assert.assertEquals(
+          "Table property must override table default properties set at catalog level.",
+          "table-key2",
+          table.properties().get("key2"));
+      Assert.assertEquals(
+          "Table property override set at catalog level must override table default" +
+              " properties set at catalog level and table property specified.",
+          "catalog-override-key3",
+          table.properties().get("key3"));
+      Assert.assertEquals(
+          "Table override not in table props or defaults should be added to table properties",
+          "catalog-override-key4",
+          table.properties().get("key4"));
+      Assert.assertEquals(
+          "Table properties without any catalog level default or override should be added to table" +
+              " properties.",
+          "table-key5",
+          table.properties().get("key5"));
+    } finally {
+      hiveCatalog.dropTable(tableIdent);
+    }
+  }
 }