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



##########
File path: core/src/main/java/org/apache/iceberg/ClientPool.java
##########
@@ -36,7 +36,7 @@
   private volatile int currentSize;
   private boolean closed;
 
-  ClientPool(int poolSize, Class<? extends E> reconnectExc) {
+  protected ClientPool(int poolSize, Class<? extends E> reconnectExc) {

Review comment:
       I don't think this is unnecessary. This base class is public now and may 
be extended outside of this package.

##########
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:
       Yeah, in this implementation, namespaces exist (if a table in the 
namespace exists) and can be listed but we don't support namespace metadata. 
That keeps things simple so that we only need one table for the initial 
implementation. In the future, we can add a separate table with custom 
metadata. But that doesn't need to be done yet.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.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.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 PROPERTY_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.URI, 
"").isEmpty(),
+        "No connection url provided for jdbc catalog!");

Review comment:
       This mixes together two separate problems. First, the URI is required so 
it must be non-null in the map. Second, the URI must be valid. This uses 
`isEmpty` to check both, but that's not sufficient to know it is a valid URI, 
and hides when the URI is not set.
   
   I think this should check that the URI is non-null and leave the validity 
check to the JDBC driver. Then this error message can be more specific:
   
   ```java
   String uri = properties.get(CatalogProperties.URI);
   Preconditions.checkNotNull(uri, "JDBC connection URI is required");
   ```

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.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.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 PROPERTY_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.URI, 
"").isEmpty(),
+        "No connection url provided for jdbc catalog!");
+
+    this.warehouseLocation = 
properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "")
+        .replaceAll("/$", "");
+    Preconditions.checkArgument(!warehouseLocation.isEmpty(),
+        "Cannot initialize Jdbc Catalog because warehousePath must not be 
null!");

Review comment:
       This should also not mix together cases where the warehouse location is 
missing vs invalid.
   
   Can you update it to this?
   
   ```java
   String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
   Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
   this.warehouseLocation = warehouse.replaceAll("/$", "");
   ```

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.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.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 PROPERTY_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.URI, 
"").isEmpty(),
+        "No connection url provided for jdbc catalog!");
+
+    this.warehouseLocation = 
properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "")
+        .replaceAll("/$", "");
+    Preconditions.checkArgument(!warehouseLocation.isEmpty(),
+        "Cannot initialize Jdbc Catalog because warehousePath must not be 
null!");
+
+    if (name != null) {
+      this.catalogName = 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);

Review comment:
       Error messages should not end in `!`

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,404 @@
+/*
+ * 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.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.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 PROPERTY_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.URI, 
"").isEmpty(),
+        "No connection url provided for jdbc catalog!");
+
+    this.warehouseLocation = 
properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "")
+        .replaceAll("/$", "");
+    Preconditions.checkArgument(!warehouseLocation.isEmpty(),
+        "Cannot initialize Jdbc Catalog because warehousePath must not be 
null!");
+
+    if (name != null) {
+      this.catalogName = 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 {

Review comment:
       I don't think it is necessary to pass all of the properties here. Just 
the URI, right?




----------------------------------------------------------------
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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to