You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by pv...@apache.org on 2022/08/04 11:03:10 UTC

[iceberg] branch master updated: Hive: Hadoop Path fails on s3 endpoint (#5405)

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

pvary 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 7f5ecad8e5 Hive: Hadoop Path fails on s3 endpoint (#5405)
7f5ecad8e5 is described below

commit 7f5ecad8e57149702cb57087fdc7c11ddacf587d
Author: Fokko Driesprong <fo...@apache.org>
AuthorDate: Thu Aug 4 13:03:04 2022 +0200

    Hive: Hadoop Path fails on s3 endpoint (#5405)
---
 .../main/java/org/apache/iceberg/hive/HiveCatalog.java  | 17 +++++++----------
 .../java/org/apache/iceberg/hive/TestHiveCatalog.java   | 14 ++++++++++++++
 2 files changed, 21 insertions(+), 10 deletions(-)

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 a86ce9dcdc..aaaa6de99a 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
@@ -24,7 +24,6 @@ import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
@@ -482,18 +481,17 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
       throw new RuntimeException("Interrupted during commit", e);
     }
 
-    // Otherwise stick to the {WAREHOUSE_DIR}/{DB_NAME}.db/{TABLE_NAME} path
-    String warehouseLocation = getWarehouseLocation();
-    return String.format(
-        "%s/%s.db/%s",
-        warehouseLocation, tableIdentifier.namespace().levels()[0], tableIdentifier.name());
+    // Otherwise, stick to the {WAREHOUSE_DIR}/{DB_NAME}.db/{TABLE_NAME} path
+    String databaseLocation = databaseLocation(tableIdentifier.namespace().levels()[0]);
+    return String.format("%s/%s", databaseLocation, tableIdentifier.name());
   }
 
-  private String getWarehouseLocation() {
+  private String databaseLocation(String databaseName) {
     String warehouseLocation = conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname);
     Preconditions.checkNotNull(
         warehouseLocation, "Warehouse location is not set: hive.metastore.warehouse.dir=null");
-    return warehouseLocation;
+    warehouseLocation = LocationUtil.stripTrailingSlash(warehouseLocation);
+    return String.format("%s/%s.db", warehouseLocation, databaseName);
   }
 
   private Map<String, String> convertToMetadata(Database database) {
@@ -518,8 +516,7 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa
     Map<String, String> parameter = Maps.newHashMap();
 
     database.setName(namespace.level(0));
-    database.setLocationUri(
-        new Path(getWarehouseLocation(), namespace.level(0)).toString() + ".db");
+    database.setLocationUri(databaseLocation(namespace.level(0)));
 
     meta.forEach(
         (key, value) -> {
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 b5040ec070..704f4ea714 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
@@ -740,4 +740,18 @@ public class TestHiveCatalog extends HiveMetastoreTest {
       hiveCatalog.dropTable(tableIdent);
     }
   }
+
+  @Test
+  public void testDatabaseLocationWithSlashInWarehouseDir() {
+    Configuration conf = new Configuration();
+    // With a trailing slash
+    conf.set("hive.metastore.warehouse.dir", "s3://bucket/");
+
+    HiveCatalog catalog = new HiveCatalog();
+    catalog.setConf(conf);
+
+    Database database = catalog.convertToDatabase(Namespace.of("database"), ImmutableMap.of());
+
+    Assert.assertEquals("s3://bucket/database.db", database.getLocationUri());
+  }
 }