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/01/06 17:05:44 UTC

[GitHub] [iceberg] jackye1995 commented on a change in pull request #1870: Iceberg Jdbc Catalog Implementation

jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r552797653



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.jdbc;
+
+import java.sql.DataTruncation;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLIntegrityConstraintViolationException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.sql.SQLWarning;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  public static final String DO_COMMIT_SQL = "UPDATE " + JdbcCatalog.SQL_TABLE_NAME +

Review comment:
       Seems like we are cross referencing constants in `JdbcCatalog` and `JdbcTableOperations`, I suggest we move all the SQL table names and statements to a centralized place such as `JdbcUtil`, what do you think?

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.jdbc;
+
+import java.io.Closeable;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLIntegrityConstraintViolationException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+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.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+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.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_CREATE_CATALOG_TABLE =
+      "CREATE TABLE " + SQL_TABLE_NAME +
+          "(catalog_name VARCHAR(1255) NOT NULL," +
+          "table_namespace VARCHAR(1255) NOT NULL," +
+          "table_name VARCHAR(1255) NOT NULL," +
+          "metadata_location VARCHAR(32768)," +
+          "previous_metadata_location VARCHAR(32768)," +
+          "PRIMARY KEY (catalog_name, table_namespace, table_name)" +
+          ")";
+  public static final String LOAD_TABLE_SQL = "SELECT * FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String LIST_TABLES_SQL = "SELECT * FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ?";
+  public static final String RENAME_TABLE_SQL = "UPDATE " + SQL_TABLE_NAME +
+      " SET table_namespace = ? , table_name = ? " +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String DROP_TABLE_SQL = "DELETE FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String GET_NAMESPACE_SQL = "SELECT table_namespace FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace LIKE ? LIMIT 1";
+  public static final String LIST_NAMESPACES_SQL = "SELECT DISTINCT table_namespace FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace LIKE ?";
+  public static final String JDBC_PARAM_PREFIX = "connection.parameter.";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private static final Joiner SLASH = Joiner.on("/");
+
+  private FileIO io;
+  private String catalogName = "jdbc";
+  private String warehouseLocation;
+  private Configuration conf;
+  private JdbcClientPool connections;
+
+  public JdbcCatalog() {
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.HIVE_URI, "").isEmpty(),
+        "No connection url provided for jdbc catalog!");
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "").isEmpty(),
+        "Cannot initialize Jdbc Catalog because warehousePath must not be null!");
+
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/$", "");
+    this.catalogName = name == null ? "jdbc" : name;
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.io = fileIOImpl == null ?
+        new HadoopFileIO(conf) :
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+
+    try {
+      initializeConnection(properties);
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Database Connection failed!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to initialize catalog!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize!", e);
+    }
+  }
+
+  private void initializeConnection(Map<String, String> properties) throws SQLException, InterruptedException {
+    LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.HIVE_URI));
+    connections = new JdbcClientPool(properties.get(CatalogProperties.HIVE_URI), properties);
+    initializeCatalogTables();
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    // need to check multiple times because some databases are using different naming standard. ex: H2db keeping
+    // table names as uppercase
+
+    boolean exists = connections.run(conn -> {
+      boolean foundTable = false;
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+      if (tables.next()) {
+        foundTable = true;
+      }
+      tables.close();
+      ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+      if (tablesUpper.next()) {
+        foundTable = true;
+      }
+      tablesUpper.close();
+      ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+      if (tablesLower.next()) {
+        foundTable = true;
+      }
+      tablesLower.close();
+      return foundTable;
+    });
+
+    // create table if not exits
+    if (!exists) {
+      connections.run(conn -> conn.prepareStatement(SQL_CREATE_CATALOG_TABLE).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, SLASH.join(table.namespace().levels()), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(DROP_TABLE_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+          sql.setString(3, identifier.name());
+          return sql.executeUpdate();
+        }
+      });
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to drop " + identifier, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.debug("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.debug("Cannot drop table: {}! table not found in the catalog.", identifier);
+      return false;
+    }
+
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+      LOG.info("Table {} data purged!", identifier);
+    }
+    return true;
+
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+
+    try {
+      return connections.run(conn -> {
+        List<TableIdentifier> results = Lists.newArrayList();
+        try (PreparedStatement sql = conn.prepareStatement(LIST_TABLES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace));
+
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            results.add(JdbcUtil.stringToTableIdentifier(rs.getString("table_namespace"), rs.getString("table_name")));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(String.format("Failed to list tables in namespace: %s", namespace), e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during JDBC operation", e);
+    }
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    try {
+      int updatedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(RENAME_TABLE_SQL)) {
+          // SET
+          sql.setString(1, JdbcUtil.namespaceToString(to.namespace()));
+          sql.setString(2, to.name());
+          // WHERE
+          sql.setString(3, catalogName);
+          sql.setString(4, JdbcUtil.namespaceToString(from.namespace()));
+          sql.setString(5, from.name());
+          return sql.executeUpdate();
+        }
+      });
+
+      if (updatedRecords == 1) {
+        LOG.debug("Successfully renamed table from {} to {}!", from, to);
+      } else if (updatedRecords == 0) {
+        throw new NoSuchTableException("Failed to rename table! Table '%s' not found in the catalog!", from);
+      } else {
+        throw new RuntimeException("Failed to rename table! Rename operation Failed");
+      }
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException("Table with name '%s' already exists in the catalog!", to);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to rename table!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to rename", e);
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogName;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {

Review comment:
       I am a bit confused, why we don't support creating namespace, but we have all the other namespace operations?
   
   Looks like the route we are going is to allow table creation with any namespace name, and simply collect namespace information by scanning the `iceberg_tables` table.
   
   I think we can do better, by having another `iceberg_namesapce` table. By doing that, we can also support storing and updating any namespace property. (I don't know if this is already discussed, I am still catching up with latest comments)

##########
File path: core/src/test/resources/simplelogger.properties
##########
@@ -0,0 +1,20 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       Is this necessary?

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.jdbc;
+
+import java.io.Closeable;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLIntegrityConstraintViolationException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+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.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+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.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_CREATE_CATALOG_TABLE =
+      "CREATE TABLE " + SQL_TABLE_NAME +
+          "(catalog_name VARCHAR(1255) NOT NULL," +
+          "table_namespace VARCHAR(1255) NOT NULL," +
+          "table_name VARCHAR(1255) NOT NULL," +
+          "metadata_location VARCHAR(32768)," +
+          "previous_metadata_location VARCHAR(32768)," +
+          "PRIMARY KEY (catalog_name, table_namespace, table_name)" +
+          ")";
+  public static final String LOAD_TABLE_SQL = "SELECT * FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String LIST_TABLES_SQL = "SELECT * FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ?";
+  public static final String RENAME_TABLE_SQL = "UPDATE " + SQL_TABLE_NAME +
+      " SET table_namespace = ? , table_name = ? " +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String DROP_TABLE_SQL = "DELETE FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String GET_NAMESPACE_SQL = "SELECT table_namespace FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace LIKE ? LIMIT 1";
+  public static final String LIST_NAMESPACES_SQL = "SELECT DISTINCT table_namespace FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace LIKE ?";
+  public static final String JDBC_PARAM_PREFIX = "connection.parameter.";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private static final Joiner SLASH = Joiner.on("/");
+
+  private FileIO io;
+  private String catalogName = "jdbc";
+  private String warehouseLocation;
+  private Configuration conf;
+  private JdbcClientPool connections;
+
+  public JdbcCatalog() {
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.HIVE_URI, "").isEmpty(),
+        "No connection url provided for jdbc catalog!");
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "").isEmpty(),
+        "Cannot initialize Jdbc Catalog because warehousePath must not be null!");
+
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/$", "");
+    this.catalogName = name == null ? "jdbc" : name;
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.io = fileIOImpl == null ?
+        new HadoopFileIO(conf) :
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+
+    try {
+      initializeConnection(properties);
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Database Connection failed!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to initialize catalog!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize!", e);
+    }
+  }
+
+  private void initializeConnection(Map<String, String> properties) throws SQLException, InterruptedException {
+    LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.HIVE_URI));
+    connections = new JdbcClientPool(properties.get(CatalogProperties.HIVE_URI), properties);
+    initializeCatalogTables();
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    // need to check multiple times because some databases are using different naming standard. ex: H2db keeping
+    // table names as uppercase
+
+    boolean exists = connections.run(conn -> {
+      boolean foundTable = false;
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+      if (tables.next()) {
+        foundTable = true;
+      }
+      tables.close();
+      ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+      if (tablesUpper.next()) {
+        foundTable = true;
+      }
+      tablesUpper.close();
+      ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+      if (tablesLower.next()) {
+        foundTable = true;
+      }
+      tablesLower.close();
+      return foundTable;
+    });
+
+    // create table if not exits
+    if (!exists) {
+      connections.run(conn -> conn.prepareStatement(SQL_CREATE_CATALOG_TABLE).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, SLASH.join(table.namespace().levels()), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(DROP_TABLE_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+          sql.setString(3, identifier.name());
+          return sql.executeUpdate();
+        }
+      });
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to drop " + identifier, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.debug("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.debug("Cannot drop table: {}! table not found in the catalog.", identifier);
+      return false;
+    }
+
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+      LOG.info("Table {} data purged!", identifier);
+    }
+    return true;
+
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+
+    try {
+      return connections.run(conn -> {
+        List<TableIdentifier> results = Lists.newArrayList();
+        try (PreparedStatement sql = conn.prepareStatement(LIST_TABLES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace));
+
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            results.add(JdbcUtil.stringToTableIdentifier(rs.getString("table_namespace"), rs.getString("table_name")));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(String.format("Failed to list tables in namespace: %s", namespace), e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during JDBC operation", e);
+    }
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    try {
+      int updatedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(RENAME_TABLE_SQL)) {
+          // SET
+          sql.setString(1, JdbcUtil.namespaceToString(to.namespace()));
+          sql.setString(2, to.name());
+          // WHERE
+          sql.setString(3, catalogName);
+          sql.setString(4, JdbcUtil.namespaceToString(from.namespace()));
+          sql.setString(5, from.name());
+          return sql.executeUpdate();
+        }
+      });
+
+      if (updatedRecords == 1) {
+        LOG.debug("Successfully renamed table from {} to {}!", from, to);
+      } else if (updatedRecords == 0) {
+        throw new NoSuchTableException("Failed to rename table! Table '%s' not found in the catalog!", from);
+      } else {
+        throw new RuntimeException("Failed to rename table! Rename operation Failed");
+      }
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException("Table with name '%s' already exists in the catalog!", to);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to rename table!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to rename", e);
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogName;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException("Cannot create namespace " + namespace +
+        ": createNamespace is not supported");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+    try {
+
+      List<Namespace> namespaces = connections.run(conn -> {
+        List<Namespace> result = Lists.newArrayList();
+
+        try (PreparedStatement sql = conn.prepareStatement(LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          if (namespace.isEmpty()) {
+            sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");

Review comment:
       why not put % in LIST_NAMESPACES_SQL? And also the same comment for other places using %.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,391 @@
+/*
+ * 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.jdbc;
+
+import java.io.Closeable;
+import java.sql.Connection;
+import java.sql.DataTruncation;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLIntegrityConstraintViolationException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.sql.SQLWarning;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+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.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+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.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + SQL_TABLE_NAME +
+                  "(catalog_name VARCHAR(1255) NOT NULL," +
+                  "table_namespace VARCHAR(1255) NOT NULL," +
+                  "table_name VARCHAR(1255) NOT NULL," +
+                  "metadata_location VARCHAR(32768)," +
+                  "previous_metadata_location VARCHAR(32768)," +
+                  "PRIMARY KEY (catalog_name, table_namespace, table_name)" +
+                  ")";
+  public static final String SQL_SELECT_TABLE = "SELECT * FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String SQL_SELECT_ALL = "SELECT * FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ?";
+  public static final String SQL_UPDATE_TABLE_NAME = "UPDATE " + SQL_TABLE_NAME +
+          " SET table_namespace = ? , table_name = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String SQL_DELETE_TABLE = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String SQL_SELECT_NAMESPACE = "SELECT table_namespace FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace LIKE ? LIMIT 1";
+  public static final String SQL_SELECT_NAMESPACES = "SELECT DISTINCT table_namespace FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace LIKE ?";
+  public static final String JDBC_PARAM_PREFIX = "connection.parameter.";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private static final Joiner SLASH = Joiner.on("/");
+
+  private FileIO fileIO;
+  private String catalogName = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private JdbcClientPool dbConnPool;
+
+  public JdbcCatalog() {
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.HIVE_URI, "").isEmpty(),
+            "No connection url provided for jdbc catalog!");
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "").isEmpty(),
+            "Cannot initialize Jdbc Catalog because warehousePath must not be null!");
+
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/$", "");
+    this.catalogName = name == null ? "jdbc" : name;
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ?
+            new HadoopFileIO(hadoopConf) :
+            CatalogUtil.loadFileIO(fileIOImpl, properties, hadoopConf);
+    initializeConnection(properties);
+  }
+
+  private void initializeConnection(Map<String, String> properties) {
+    try {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.HIVE_URI));
+      Properties dbProps = new Properties();
+      for (Map.Entry<String, String> prop : properties.entrySet()) {
+        if (prop.getKey().startsWith(JDBC_PARAM_PREFIX)) {
+          dbProps.put(prop.getKey().substring(JDBC_PARAM_PREFIX.length()), prop.getValue());
+        }
+      }
+      dbConnPool = new JdbcClientPool(properties.get(CatalogProperties.HIVE_URI), dbProps);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Database Connection failed!", e);
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException("Database connection warning!", e);
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to initialize Jdbc Catalog!", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    // need to check multiple times because some databases are using different naming standard. ex: H2db keeping
+    // table names as uppercase

Review comment:
       SQL standard requires names stored in uppercase. I think we can just use upper case names for all tables we define.
   
   nit: put ex on a second line as a complete sentence

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,423 @@
+/*
+ * 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.jdbc;
+
+import java.io.Closeable;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLIntegrityConstraintViolationException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+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.iceberg.BaseMetastoreCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+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.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_CREATE_CATALOG_TABLE =
+      "CREATE TABLE " + SQL_TABLE_NAME +
+          "(catalog_name VARCHAR(1255) NOT NULL," +
+          "table_namespace VARCHAR(1255) NOT NULL," +
+          "table_name VARCHAR(1255) NOT NULL," +
+          "metadata_location VARCHAR(32768)," +
+          "previous_metadata_location VARCHAR(32768)," +
+          "PRIMARY KEY (catalog_name, table_namespace, table_name)" +
+          ")";
+  public static final String LOAD_TABLE_SQL = "SELECT * FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String LIST_TABLES_SQL = "SELECT * FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ?";
+  public static final String RENAME_TABLE_SQL = "UPDATE " + SQL_TABLE_NAME +
+      " SET table_namespace = ? , table_name = ? " +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String DROP_TABLE_SQL = "DELETE FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+  public static final String GET_NAMESPACE_SQL = "SELECT table_namespace FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace LIKE ? LIMIT 1";
+  public static final String LIST_NAMESPACES_SQL = "SELECT DISTINCT table_namespace FROM " + SQL_TABLE_NAME +
+      " WHERE catalog_name = ? AND table_namespace LIKE ?";
+  public static final String JDBC_PARAM_PREFIX = "connection.parameter.";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private static final Joiner SLASH = Joiner.on("/");
+
+  private FileIO io;
+  private String catalogName = "jdbc";
+  private String warehouseLocation;
+  private Configuration conf;
+  private JdbcClientPool connections;
+
+  public JdbcCatalog() {
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.HIVE_URI, "").isEmpty(),
+        "No connection url provided for jdbc catalog!");
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "").isEmpty(),
+        "Cannot initialize Jdbc Catalog because warehousePath must not be null!");
+
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/$", "");
+    this.catalogName = name == null ? "jdbc" : name;
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.io = fileIOImpl == null ?
+        new HadoopFileIO(conf) :
+        CatalogUtil.loadFileIO(fileIOImpl, properties, conf);
+
+    try {
+      initializeConnection(properties);
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Database Connection failed!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to initialize catalog!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize!", e);
+    }
+  }
+
+  private void initializeConnection(Map<String, String> properties) throws SQLException, InterruptedException {
+    LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.HIVE_URI));
+    connections = new JdbcClientPool(properties.get(CatalogProperties.HIVE_URI), properties);
+    initializeCatalogTables();
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    // need to check multiple times because some databases are using different naming standard. ex: H2db keeping
+    // table names as uppercase
+
+    boolean exists = connections.run(conn -> {
+      boolean foundTable = false;
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+      if (tables.next()) {
+        foundTable = true;
+      }
+      tables.close();
+      ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+      if (tablesUpper.next()) {
+        foundTable = true;
+      }
+      tablesUpper.close();
+      ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+      if (tablesLower.next()) {
+        foundTable = true;
+      }
+      tablesLower.close();
+      return foundTable;
+    });
+
+    // create table if not exits
+    if (!exists) {
+      connections.run(conn -> conn.prepareStatement(SQL_CREATE_CATALOG_TABLE).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, SLASH.join(table.namespace().levels()), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(DROP_TABLE_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+          sql.setString(3, identifier.name());
+          return sql.executeUpdate();
+        }
+      });
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to drop " + identifier, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.debug("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.debug("Cannot drop table: {}! table not found in the catalog.", identifier);
+      return false;
+    }
+
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+
+    if (purge && lastMetadata != null) {
+      CatalogUtil.dropTableData(ops.io(), lastMetadata);
+      LOG.info("Table {} data purged!", identifier);
+    }
+    return true;
+
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+
+    try {
+      return connections.run(conn -> {
+        List<TableIdentifier> results = Lists.newArrayList();
+        try (PreparedStatement sql = conn.prepareStatement(LIST_TABLES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace));
+
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            results.add(JdbcUtil.stringToTableIdentifier(rs.getString("table_namespace"), rs.getString("table_name")));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(String.format("Failed to list tables in namespace: %s", namespace), e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during JDBC operation", e);
+    }
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    try {
+      int updatedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(RENAME_TABLE_SQL)) {
+          // SET
+          sql.setString(1, JdbcUtil.namespaceToString(to.namespace()));
+          sql.setString(2, to.name());
+          // WHERE
+          sql.setString(3, catalogName);
+          sql.setString(4, JdbcUtil.namespaceToString(from.namespace()));
+          sql.setString(5, from.name());
+          return sql.executeUpdate();
+        }
+      });
+
+      if (updatedRecords == 1) {
+        LOG.debug("Successfully renamed table from {} to {}!", from, to);
+      } else if (updatedRecords == 0) {
+        throw new NoSuchTableException("Failed to rename table! Table '%s' not found in the catalog!", from);
+      } else {
+        throw new RuntimeException("Failed to rename table! Rename operation Failed");
+      }
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException("Table with name '%s' already exists in the catalog!", to);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to rename table!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to rename", e);
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogName;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void createNamespace(Namespace namespace, Map<String, String> metadata) {
+    throw new UnsupportedOperationException("Cannot create namespace " + namespace +
+        ": createNamespace is not supported");
+  }
+
+  @Override
+  public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespaceException {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+    try {
+
+      List<Namespace> namespaces = connections.run(conn -> {
+        List<Namespace> result = Lists.newArrayList();
+
+        try (PreparedStatement sql = conn.prepareStatement(LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          if (namespace.isEmpty()) {
+            sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          } else {
+            sql.setString(2, JdbcUtil.namespaceToString(namespace) + ".%");
+          }
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString("table_namespace")));
+          }
+          rs.close();
+        }
+
+        return result;
+      });
+
+      int subNamespaceLevelLength = namespace.levels().length + 1;
+      namespaces = namespaces.stream()
+          // exclude itself
+          .filter(n -> !n.equals(namespace))
+          // only get sub namespaces/children
+          .map(n -> Namespace.of(
+              Arrays.stream(n.levels()).limit(subNamespaceLevelLength).toArray(String[]::new)
+              )
+          )
+          // remove duplicates
+          .distinct()
+          .collect(Collectors.toList());
+
+      LOG.debug("From the namespace '{}' found: {}", namespace, namespaces);
+      return namespaces;
+
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to listNamespaces(namespace) Namespace: " + namespace, e);
+    } catch (SQLException e) {
+      throw new RuntimeException("Failed to list all namespace: " + namespace + " in catalog!", e);
+    }
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    if (!namespaceExists(namespace) || namespace.isEmpty()) {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+
+    Path nsPath = new Path(warehouseLocation, SLASH.join(namespace.levels()));

Review comment:
       nit: seems like we are just using `Path` to join the two paths. Why not just use `SLASH.join(warehouseLocation, SLASH.join(namespace.levels()))`
   
   Also, if this is the default namespace path, I think it is better to have its own method:
   
   ```
   private String defaultNamespaceLocation() {
       return ....
   }




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

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