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 2020/12/03 15:27:17 UTC

[GitHub] [iceberg] ismailsimsek opened a new pull request #1870: Iceberg Jdbc Catalog Implementation

ismailsimsek opened a new pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870


   Implemented simple Jdbc Catalog #1634 it useful feature to have for some scenarios
   
   cc: @kbendick @jackye1995 
   
   closes #1634 


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547575767



##########
File path: core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java
##########
@@ -0,0 +1,566 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.Transaction;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.transforms.Transform;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.NullOrder.NULLS_FIRST;
+import static org.apache.iceberg.SortDirection.ASC;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestJdbcCatalog {

Review comment:
       I think that there need to be tests for the JDBC operations that are exposed by `JdbcTable` directly. Those should check cases that are required, like creating a table fails if it already exists, updating a table when metadata is out of date catches the error, etc. Then we can add functional tests to ensure that the catalog calls those correctly.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r548237430



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {

Review comment:
       this was good idea implement it, and reorganized code a bit 
   >  I think an improvement would be to instantiate one per catalog instead (and pass in a connection pool so it can be shared with TableOperations). You'd probably want to rename it to something like TableSQL?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r553459779



##########
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:
       this one is database dependent. following section is looking for a table case sensitive(database dependent)
   if we use `CREATE TABLE IF NOT EXISTS` we can remove this method, [more details here ](https://github.com/apache/iceberg/pull/1870#discussion_r550468895)
   ```
   DatabaseMetaData dbMeta = conn.getMetaData();
   ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
   ```
   
   




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r562913098



##########
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:
       important parameters passed with properties are `clients` pool size and `username` `password`. except client pool size all other db parameters could be passed by uri .




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


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

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536413936



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;

Review comment:
       The `Configurable` interface that this implements is part of `org.apache.hadoop.conf`. Though I somewhat still agree. In my implementation, I don't have it in this package.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r562300476



##########
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r553473774



##########
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:
       moved sql constants to `JdbcUtil`




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r548265410



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return !this.get(tableIdentifier).isEmpty();
+    } catch (SQLException | JsonProcessingException throwables) {
+      return false;
+    }
+  }
+
+  public Map<String, String> get(TableIdentifier tableIdentifier) throws SQLException, JsonProcessingException {
+    Map<String, String> table = Maps.newHashMap();
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+    sql.setString(3, tableIdentifier.name());
+    ResultSet rs = sql.executeQuery();
+    if (rs.next()) {
+      table.put("catalog_name", rs.getString("catalog_name"));
+      table.put("table_namespace", rs.getString("table_namespace"));
+      table.put("table_name", rs.getString("table_name"));
+      table.put("metadata_location", rs.getString("metadata_location"));
+      table.put("previous_metadata_location", rs.getString("previous_metadata_location"));
+    }
+    rs.close();
+    return table;
+  }
+
+  public List<TableIdentifier> getAll(Namespace namespace) throws SQLException, JsonProcessingException {
+    List<TableIdentifier> results = Lists.newArrayList();
+
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT_ALL);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(namespace));
+    ResultSet rs = sql.executeQuery();
+
+    while (rs.next()) {
+      final TableIdentifier table = JdbcUtil.stringToTableIdentifier(
+              rs.getString("table_namespace"), rs.getString("table_name"));
+      results.add(table);
+    }
+    rs.close();
+    return results;
+  }
+
+  public int updateMetadataLocation(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                                    String newMetadataLocation) throws SQLException, JsonProcessingException {
+    PreparedStatement sql = dbConn.prepareStatement(SQL_UPDATE_METADATA_LOCATION);
+    // UPDATE
+    sql.setString(1, newMetadataLocation);
+    sql.setString(2, oldMetadataLocation);
+    // WHERE
+    sql.setString(3, catalogName);
+    sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+    sql.setString(5, tableIdentifier.name());
+    sql.setString(6, oldMetadataLocation);
+    return sql.executeUpdate();
+  }
+
+  public int save(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                  String newMetadataLocation) throws SQLException, JsonProcessingException {

Review comment:
       removed




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r548238230



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  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 name = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection dbConn;

Review comment:
       moved it to core and used for jdbc catalog




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


[GitHub] [iceberg] rdblue commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-863595846






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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r654811816



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);

Review comment:
       it was stuck in infinite loop(calling .refresh) so i had to call disableRefresh(); before the return
   
   loop call was
   ```
   	at org.apache.iceberg.BaseMetastoreTableOperations.current(BaseMetastoreTableOperations.java:77)
   	at org.apache.iceberg.BaseMetastoreTableOperations.refresh(BaseMetastoreTableOperations.java:106)
   ```
   
   ```
   
       if (table.isEmpty()) {
         if (currentMetadataLocation() != null) {
           throw new NoSuchTableException("Failed to load table %s from catalog %s: dropped by another process",
               tableIdentifier, catalogName);
         } else {
           this.disableRefresh(); # new method added to `BaseMetastoreTableOperations` sets shouldRefresh=false
           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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637618765



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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(JdbcUtil.LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.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
+          .filter(n -> n.levels().length >= subNamespaceLevelLength)
+          // 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);
+    }
+
+    return ImmutableMap.of("location", defaultNamespaceLocation(namespace));
+  }
+
+  private String defaultNamespaceLocation(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return warehouseLocation;
+    } else {
+      return SLASH.join(warehouseLocation, SLASH.join(namespace.levels()));
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Cannot drop namespace %s because it is not found!", namespace);
+    }
+
+    List<TableIdentifier> tableIdentifiers = listTables(namespace);
+    if (tableIdentifiers != null && !tableIdentifiers.isEmpty()) {
+      throw new NamespaceNotEmptyException("Cannot drop namespace %s because it is not empty. " +

Review comment:
       No need to list tables, and `because` can be replaced with `:`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637619004



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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(JdbcUtil.LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.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
+          .filter(n -> n.levels().length >= subNamespaceLevelLength)
+          // 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);
+    }
+
+    return ImmutableMap.of("location", defaultNamespaceLocation(namespace));
+  }
+
+  private String defaultNamespaceLocation(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return warehouseLocation;
+    } else {
+      return SLASH.join(warehouseLocation, SLASH.join(namespace.levels()));
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Cannot drop namespace %s because it is not found!", namespace);
+    }
+
+    List<TableIdentifier> tableIdentifiers = listTables(namespace);
+    if (tableIdentifiers != null && !tableIdentifiers.isEmpty()) {
+      throw new NamespaceNotEmptyException("Cannot drop namespace %s because it is not empty. " +
+          "Namespace contains %s tables", namespace, tableIdentifiers.size());
+    }
+
+    // namespaces are created/deleted by tables by default return true
+    // when there is no tables with namespace then its considered dropped
+    return true;

Review comment:
       This conflicts with the stated behavior from the API. If no action is taken, the method should return `false`: https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/catalog/SupportsNamespaces.java#L102




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536894519



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);

Review comment:
       We typically pass a connection string via [`uri`](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/CatalogProperties.java#L31).
   
   It may also be a good idea to have a way to avoid exposing credentials. We typically allow registering a `Supplier<String>` that is called, although here it may make sense to use `Function<Map<String, String>, String>` so that the catalog config can be used.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549524891



##########
File path: api/src/main/java/org/apache/iceberg/exceptions/UncheckedSQLException.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.exceptions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.SQLException;
+
+public class UncheckedSQLException extends UncheckedIOException {
+
+  public UncheckedSQLException(String message, SQLException cause) {
+    super(message, new IOException(cause));
+  }
+
+  public UncheckedSQLException(String message, Exception cause) {
+    super(message, new IOException(cause));
+  }
+
+  public UncheckedSQLException(String message, InterruptedException cause) {
+    super(message, new IOException(cause));
+  }
+
+  public UncheckedSQLException(String message) {

Review comment:
       I don't think this is needed.

##########
File path: api/src/main/java/org/apache/iceberg/exceptions/UncheckedSQLException.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.exceptions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.SQLException;
+
+public class UncheckedSQLException extends UncheckedIOException {
+
+  public UncheckedSQLException(String message, SQLException cause) {
+    super(message, new IOException(cause));
+  }
+
+  public UncheckedSQLException(String message, Exception cause) {
+    super(message, new IOException(cause));
+  }
+
+  public UncheckedSQLException(String message, InterruptedException cause) {

Review comment:
       I don't think this is needed.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641567328



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);

Review comment:
       created `UncheckedInterruptedException` and replaced all




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641543713



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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(JdbcUtil.LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.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
+          .filter(n -> n.levels().length >= subNamespaceLevelLength)
+          // 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);
+    }
+
+    return ImmutableMap.of("location", defaultNamespaceLocation(namespace));
+  }
+
+  private String defaultNamespaceLocation(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return warehouseLocation;
+    } else {
+      return SLASH.join(warehouseLocation, SLASH.join(namespace.levels()));
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Cannot drop namespace %s because it is not found!", namespace);
+    }
+
+    List<TableIdentifier> tableIdentifiers = listTables(namespace);
+    if (tableIdentifiers != null && !tableIdentifiers.isEmpty()) {
+      throw new NamespaceNotEmptyException("Cannot drop namespace %s because it is not empty. " +
+          "Namespace contains %s tables", namespace, tableIdentifiers.size());
+    }
+
+    // namespaces are created/deleted by tables by default return true
+    // when there is no tables with namespace then its considered dropped
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws
+      NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void close() {
+    connections.close();
+  }
+
+  @Override
+  public boolean namespaceExists(Namespace namespace) {
+    try {
+      return connections.run(conn -> {
+        boolean exists = false;
+
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.GET_NAMESPACE_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          if (rs.next()) {
+            exists = true;
+          }
+
+          rs.close();
+        }
+
+        return exists;
+      });
+
+    } catch (SQLException e) {
+      LOG.warn("SQLException! ", e);
+      return false;

Review comment:
       replaced with  `throw new UncheckedSQLException(e, "Failed to get namespace %s", namespace);`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536897555



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcNamespaceDao.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcNamespaceDao {
+  public static final String NAMESPACES_TABLE_NAME = "iceberg_namespaces";
+  public static final String NAMESPACES_TABLE_DDL = "CREATE TABLE IF NOT EXISTS " +
+          JdbcNamespaceDao.NAMESPACES_TABLE_NAME + " ( " +
+          "catalogName VARCHAR(1255) NOT NULL," +
+          "namespace VARCHAR(1255) NOT NULL," +
+          "namespaceMetadata VARCHAR(32768)," +
+          "PRIMARY KEY (catalogName, namespace) " +
+          ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcNamespaceDao.class);
+  private final String catalogName;
+  private QueryRunner queryRunner;
+
+
+  public JdbcNamespaceDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public JdbcNamespace get(Namespace namespace) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + NAMESPACES_TABLE_NAME + " WHERE catalogName = " +
+                    "? AND namespace = ? ;",
+            new BeanHandler<>(JdbcNamespace.class), catalogName, namespace.toString());
+  }
+
+  public boolean isExists(Namespace namespace) {
+    try {
+      return this.get(namespace) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public List<JdbcNamespace> getAll() throws SQLException {

Review comment:
       You may want to also have a prefix-based lookup using LIKE:
   
   ```java
   String namespacePrefix = namespace.toString() + ".%";
   ```
   ```sql
   SELECT * FROM iceberg_namespaces WHERE catalog_name = ? AND namespace LIKE ?
   ```




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536325774



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;

Review comment:
       Why put this in the `hadoop` package? There isn't anything specific to Hadoop is there?




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


[GitHub] [iceberg] rdblue commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-761291374


   I'm hoping to be able to review this over the weekend. Thanks for your patience, @ismailsimsek!


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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641499557



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);

Review comment:
       aligned log messages with hive and removed `!`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637616599



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");

Review comment:
       In `HadoopCatalog`, this is `/*$`. Maybe we should do the same here?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536904152



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcNamespace.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iceberg.catalog.Namespace;
+
+public class JdbcNamespace {
+  private String catalogName;
+  private String namespace;
+  private String namespaceMetadata;
+
+  public JdbcNamespace(Namespace namespace) {
+    this.namespace = namespace.toString();
+  }
+
+  public String getCatalogName() {
+    return catalogName;
+  }
+
+  public void setCatalogName(String catalogName) {
+    this.catalogName = catalogName;
+  }
+
+  public String getNamespace() {
+    return namespace;
+  }
+
+  public void setNamespace(String namespace) {
+    this.namespace = namespace;
+  }
+
+  public String getNamespaceMetadata() {
+    return namespaceMetadata;
+  }
+
+  public void setNamespaceMetadata(String namespaceMetadata) {
+    this.namespaceMetadata = namespaceMetadata;
+  }
+
+  public Namespace toNamespace() {
+    return Namespace.of(namespaceMetadata);
+  }
+
+  public Map<String, String> toNamespaceMetadata() throws JsonProcessingException {

Review comment:
       For the initial version, let's not support namespace metadata. We can add that later in a separate PR where we consider more options for storage.




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


[GitHub] [iceberg] rdblue commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-739100512


   Thanks for working on this, @ismailsimsek! I'll try to take a close look soon.
   
   The only thing I noticed right away was the use of a few new libraries. I'd really prefer to keep dependencies to a minimum, so please consider what is doing heavy work vs. what is just a convenience.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536896938



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {

Review comment:
       Any SQL exception? What possible exceptions does this include?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549524869



##########
File path: api/src/main/java/org/apache/iceberg/exceptions/UncheckedSQLException.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.exceptions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.SQLException;
+
+public class UncheckedSQLException extends UncheckedIOException {
+
+  public UncheckedSQLException(String message, SQLException cause) {
+    super(message, new IOException(cause));
+  }
+
+  public UncheckedSQLException(String message, Exception cause) {

Review comment:
       There is no need to wrap an exception that isn't a `SQLException` is there?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537727662



##########
File path: build.gradle
##########
@@ -231,6 +231,7 @@ project(':iceberg-core') {
       exclude group: 'org.slf4j', module: 'slf4j-log4j12'
     }
 
+    testCompile "com.h2database:h2"

Review comment:
       Should this be `testRuntime` because the only compile dependency is for JDBC?
   
   Why switch databases?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r553433934



##########
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:
       changed to `SLASH.join` and created method `defaultNamespaceLocation`




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r556306959



##########
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:
       nit: unnecessary keyword `protected`

##########
File path: api/src/main/java/org/apache/iceberg/exceptions/UncheckedSQLException.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.exceptions;
+
+import java.sql.SQLException;
+
+public class UncheckedSQLException extends RuntimeException {

Review comment:
       I think this exception is specific to JDBC catalog and should be in the core jdbc package path.

##########
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.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 {
+
+  protected static final String JDBC_PARAM_PREFIX = "connection.parameter.";

Review comment:
       1. This needs to be public, so that users can import this variable to set their custom properties.
   2. to be consistent with the name `CatalogProperties`, this should be named `PROPERTY_PREFIX`, so that users can use `JdbcCatalog.PROPERTY_PREFIX`.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    } catch (SQLException e) {
+      // unknown exception happened when getting table from catalog
+      throw new UncheckedSQLException(String.format("Failed to get table from catalog %s.%s", catalogName,
+          tableIdentifier), e);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table from catalog %s.%s!" +
+          " maybe another process deleted it!", catalogName, tableIdentifier);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault("metadata_location", null) == null) {

Review comment:
       "metadata_location" should be a static variable

##########
File path: build.gradle
##########
@@ -231,6 +231,7 @@ project(':iceberg-core') {
       exclude group: 'org.slf4j', module: 'slf4j-log4j12'
     }
 
+    testCompile "com.h2database:h2"

Review comment:
       The MPL license is a copyleft license, which means that in principle people are not allowed to distribute code that is under the MPL-2.0 license under different terms. The GPL licenses (including LGPL and AGPL) require that the entire application is distributed under the terms of the GPL license.
   
   I think we cannot use it. Could you switch to use https://mvnrepository.com/artifact/org.xerial/sqlite-jdbc?

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    } catch (SQLException e) {
+      // unknown exception happened when getting table from catalog
+      throw new UncheckedSQLException(String.format("Failed to get table from catalog %s.%s", catalogName,
+          tableIdentifier), e);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table from catalog %s.%s!" +
+          " maybe another process deleted it!", catalogName, tableIdentifier);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault("metadata_location", null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location if the table %s.%s", catalogName,
+          tableIdentifier));
+    }
+
+    refreshFromMetadataLocation(table.get("metadata_location"));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {

Review comment:
       method too long, prefer to separate to smaller methods like `updateTable` and `createTable` based on exist condition.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientConnectionException;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+
+public class JdbcClientPool extends ClientPool<Connection, SQLException> {

Review comment:
       This do not need to be public

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    } catch (SQLException e) {
+      // unknown exception happened when getting table from catalog
+      throw new UncheckedSQLException(String.format("Failed to get table from catalog %s.%s", catalogName,

Review comment:
       nit: `Failed to get table %s from catalog %s, tableIdentifier, catalogName` sounds better, also for other similar error messages below.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientConnectionException;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+
+public class JdbcClientPool extends ClientPool<Connection, SQLException> {
+
+  private final String dbUrl;
+  private final Map<String, String> properties;
+
+  JdbcClientPool(String dbUrl, Map<String, String> props) {
+    this(Integer.parseInt(props.getOrDefault(CatalogProperties.HIVE_CLIENT_POOL_SIZE,
+        String.valueOf(CatalogProperties.HIVE_CLIENT_POOL_SIZE_DEFAULT))), dbUrl, props);
+  }
+
+  public JdbcClientPool(int poolSize, String dbUrl, Map<String, String> props) {
+    super(poolSize, SQLNonTransientConnectionException.class);
+    properties = props;
+    this.dbUrl = dbUrl;
+  }
+
+  @Override
+  protected Connection newClient() {
+    try {
+      Properties dbProps = new Properties();
+      properties.forEach((key, value) -> dbProps.put(key.replace(JdbcCatalog.JDBC_PARAM_PREFIX, ""), value));
+      return DriverManager.getConnection(dbUrl, dbProps);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to connect: " + dbUrl, e);
+    }
+  }
+
+  @Override
+  protected Connection reconnect(Connection client) {
+    close(client);
+    return newClient();
+  }
+
+  @Override
+  protected void close(Connection client) {
+    try {
+      client.close();
+    } catch (SQLException e) {
+      throw new RuntimeException("Failed to connect to database!", e);

Review comment:
       why not use your `UncheckedSQLException`?

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientConnectionException;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+
+public class JdbcClientPool extends ClientPool<Connection, SQLException> {
+
+  private final String dbUrl;
+  private final Map<String, String> properties;
+
+  JdbcClientPool(String dbUrl, Map<String, String> props) {
+    this(Integer.parseInt(props.getOrDefault(CatalogProperties.HIVE_CLIENT_POOL_SIZE,
+        String.valueOf(CatalogProperties.HIVE_CLIENT_POOL_SIZE_DEFAULT))), dbUrl, props);
+  }
+
+  public JdbcClientPool(int poolSize, String dbUrl, Map<String, String> props) {

Review comment:
       This do not need to be public




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


[GitHub] [iceberg] rdblue merged pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870


   


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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637402262



##########
File path: core/src/main/java/org/apache/iceberg/CatalogUtil.java
##########
@@ -48,8 +48,10 @@
   public static final String ICEBERG_CATALOG_TYPE = "type";
   public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop";
   public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive";
+  public static final String ICEBERG_CATALOG_TYPE_JDBC = "jdbc";

Review comment:
       makes sense, rolled the change back




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536896874



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public JdbcTable get(TableIdentifier tableIdentifier) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ",

Review comment:
       For structure, I would prefer to have all of the SQL statements as constants up at the top of the file, like the `CREATE TABLE` command. That makes it easy to read them independent of use here.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537120280



##########
File path: build.gradle
##########
@@ -226,6 +226,10 @@ project(':iceberg-core') {
     compile "com.fasterxml.jackson.core:jackson-databind"
     compile "com.fasterxml.jackson.core:jackson-core"
     compile "com.github.ben-manes.caffeine:caffeine"
+    compile 'commons-dbutils:commons-dbutils:1.7'
+    compile "org.hsqldb:hsqldb:2.5.1"
+    compile "mysql:mysql-connector-java:8.0.21"
+    compile "org.postgresql:postgresql:42.2.18"

Review comment:
       changed to [h2dataabse](https://www.h2database.com) and added it to LICENSE file. its  [Mozilla Public License Version 2.0](https://www.h2database.com/html/license.html) it seems compatible once we include it in LICENSE file 
   




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537047495



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);

Review comment:
       changed to `DriverManager` and `Properties`. creating  Properties using prefix `jdbccatalog.property.`
   




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537601980



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +

Review comment:
       `PRIMARY KEY (catalogName,tableNamespace,tableName)` is implemented in most relational databases. 
   but second one might not be. its nice to have PK to avoid duplicate data. another option might be adding it with alter statement it shuld be supported by many dbs 
   `ALTER TABLE my_table ADD PRIMARY KEY (columnx);`
   
   removed `PRIMARY KEY` and `FOREIGN KEY` lines for now.




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r557603107



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.HIVE_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 {
+    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 {
+    connections.run(conn -> {
+      boolean catalogTableExists = false;
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tablesUpper = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME.toUpperCase(), null);

Review comment:
       nit: since we removed lower case part, there is not need to have `upper` in the variable names, and no need to use `JdbcUtil.CATALOG_TABLE_NAME.toUpperCase()`

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    } catch (SQLException e) {
+      // unknown exception happened when getting table from catalog
+      throw new UncheckedSQLException(
+          String.format("Failed to get table %s from catalog %s", tableIdentifier, catalogName), e);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s!" +
+          " maybe another process deleted it!", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it!
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists! maybe another process created it!");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Database Connection failed!", e);
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException("Database data truncation error!", e);
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException("Database warning!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to connect to database!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during commit", e);
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to commit the table %s from catalog %s! " +

Review comment:
       nit: fail to update the table

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.HIVE_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;
+    }

Review comment:
       nit: space after if

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public final class JdbcUtil {

Review comment:
       no need to be public




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549528416



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/SparkCatalog.java
##########
@@ -111,14 +113,20 @@ protected Catalog buildIcebergCatalog(String name, CaseInsensitiveStringMap opti
     switch (catalogType.toLowerCase(Locale.ENGLISH)) {
       case ICEBERG_CATALOG_TYPE_HIVE:
         int clientPoolSize = options.getInt(CatalogProperties.HIVE_CLIENT_POOL_SIZE,
-            CatalogProperties.HIVE_CLIENT_POOL_SIZE_DEFAULT);
+                CatalogProperties.HIVE_CLIENT_POOL_SIZE_DEFAULT);

Review comment:
       Nit: unnecessary whitespace change.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549527276



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.sql.SQLWarning;
+import java.util.Properties;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+
+public class JdbcClientPool extends ClientPool<Connection, SQLException> {
+
+  private final String dbUrl;
+  private final Properties dbProperties;
+
+  JdbcClientPool(String dbUrl, Properties props) {
+    this((Integer) props.getOrDefault("iceberg.jdbc.client-pool-size", 5), dbUrl, props);
+  }
+
+  public JdbcClientPool(int poolSize, String dbUrl, Properties props) {
+    super(poolSize, SQLException.class);

Review comment:
       The second argument for the `ClientPool` constructor is an exception that indicates a connection has gone bad and should be recreated using `reconnect`. I don't think that any `SQLException` should trigger reconnecting because there are a lot of different subclasses that have nothing to do with connection issues.
   
   In Hive, we use `TTransportException`, which indicates a connection failure rather than an error from the remote service. It looks like the equivalent for JDBC might be `SQLNonTransientConnectionException`.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547568828



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  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 name = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection dbConn;
+
+  public JdbcCatalog() {
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  @Override
+  public void initialize(String name, Map<String, String> properties) throws UncheckedIOException {
+    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.name = 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) throws UncheckedIOException {
+    try {
+
+      LOG.debug("Connecting to Jdbc database {}.", properties.get(CatalogProperties.HIVE_URI));
+      Properties connectionProps = new Properties();
+      for (Map.Entry<String, String> prop : properties.entrySet()) {
+        if (prop.getKey().startsWith(JDBC_PARAM_PREFIX)) {
+          connectionProps.put(prop.getKey().substring(JDBC_PARAM_PREFIX.length()), prop.getValue());
+        }
+      }
+
+      dbConn = DriverManager.getConnection(properties.get(CatalogProperties.HIVE_URI), connectionProps);
+      initializeCatalogTables();
+
+    } catch (SQLException throwables) {
+      throw new UncheckedIOException("Failed to initialize Jdbc Catalog!", new IOException(throwables));

Review comment:
       No need to wrap in an `IOException`. I think we should create an unchecked exception used to wrap `SQLException`.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r540077765



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public JdbcTable get(TableIdentifier tableIdentifier) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ",
+            new BeanHandler<>(JdbcTable.class),
+            catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());
+  }
+
+  public List<TableIdentifier> getAll(Namespace namespace) throws SQLException {
+    List<JdbcTable> tables = queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ?",
+            new BeanListHandler<>(JdbcTable.class), catalogName, namespace.toString());
+    List<TableIdentifier> results = Lists.newArrayList();
+    for (JdbcTable table : tables) {
+      results.add(table.toTableIdentifier());
+    }
+    return results;
+  }
+
+  public void update(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                     String newMetadataLocation) throws SQLException {
+    queryRunner.update("UPDATE " + TABLES_TABLE_NAME + " SET metadataLocation = ? , previousMetadataLocation = ? " +
+                    "WHERE catalogName = ? AND tableNamespace = ? " +
+                    "AND tableName = ? ", newMetadataLocation,
+            oldMetadataLocation, catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());

Review comment:
       updated the logic, now it will 
   for committing table: 
   - succeed only if it effects row =1 
   - else rollback with `CommitFailedException`
   
   for renaming table: 
   - succeed only if it effects row = 1 
   - rollback with `NoSuchTableException`  if it effects row =0
   - else rollback fail with `UncheckedIOException` 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637618664



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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(JdbcUtil.LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.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
+          .filter(n -> n.levels().length >= subNamespaceLevelLength)
+          // 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()) {

Review comment:
       `defaultNamespaceLocation` supports empty namespaces. Why did you choose to reject empty here?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547566183



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return !this.get(tableIdentifier).isEmpty();
+    } catch (SQLException | JsonProcessingException throwables) {
+      return false;
+    }
+  }
+
+  public Map<String, String> get(TableIdentifier tableIdentifier) throws SQLException, JsonProcessingException {
+    Map<String, String> table = Maps.newHashMap();
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+    sql.setString(3, tableIdentifier.name());
+    ResultSet rs = sql.executeQuery();
+    if (rs.next()) {
+      table.put("catalog_name", rs.getString("catalog_name"));
+      table.put("table_namespace", rs.getString("table_namespace"));
+      table.put("table_name", rs.getString("table_name"));
+      table.put("metadata_location", rs.getString("metadata_location"));
+      table.put("previous_metadata_location", rs.getString("previous_metadata_location"));
+    }
+    rs.close();
+    return table;
+  }
+
+  public List<TableIdentifier> getAll(Namespace namespace) throws SQLException, JsonProcessingException {
+    List<TableIdentifier> results = Lists.newArrayList();
+
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT_ALL);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(namespace));
+    ResultSet rs = sql.executeQuery();
+
+    while (rs.next()) {
+      final TableIdentifier table = JdbcUtil.stringToTableIdentifier(
+              rs.getString("table_namespace"), rs.getString("table_name"));
+      results.add(table);
+    }
+    rs.close();
+    return results;
+  }
+
+  public int updateMetadataLocation(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                                    String newMetadataLocation) throws SQLException, JsonProcessingException {
+    PreparedStatement sql = dbConn.prepareStatement(SQL_UPDATE_METADATA_LOCATION);
+    // UPDATE
+    sql.setString(1, newMetadataLocation);
+    sql.setString(2, oldMetadataLocation);
+    // WHERE
+    sql.setString(3, catalogName);
+    sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+    sql.setString(5, tableIdentifier.name());
+    sql.setString(6, oldMetadataLocation);
+    return sql.executeUpdate();
+  }
+
+  public int save(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                  String newMetadataLocation) throws SQLException, JsonProcessingException {

Review comment:
       Why are there two metadata locations passed here?




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


[GitHub] [iceberg] rdblue commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-839985487


   Thanks, @ismailsimsek! I'll take another look. Sorry for the delay, I've been OOO for a little while.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536906293



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableOperations.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.Objects;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private QueryRunner queryRunner;
+
+  protected JdbcTableOperations(QueryRunner queryRunner, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    String metadataLocation = null;
+    JdbcTableDao tableDao = new JdbcTableDao(queryRunner, catalogName);
+    JdbcTable table = null;
+    try {
+      table = tableDao.get(tableIdentifier);
+    } catch (SQLException throwables) {
+      LOG.debug("Table not found: {} , {}!", tableIdentifier, throwables);

Review comment:
       This doesn't look quite right because the exception is too generic to simply ignore and move on.
   
   Also, no need to add a format argument for the exception. SLF4J loggers will detect that the last argument is an exception and add it properly without a placeholder for it.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536895473



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public JdbcTable get(TableIdentifier tableIdentifier) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ",
+            new BeanHandler<>(JdbcTable.class),
+            catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());
+  }
+
+  public List<TableIdentifier> getAll(Namespace namespace) throws SQLException {
+    List<JdbcTable> tables = queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ?",
+            new BeanListHandler<>(JdbcTable.class), catalogName, namespace.toString());
+    List<TableIdentifier> results = Lists.newArrayList();
+    for (JdbcTable table : tables) {
+      results.add(table.toTableIdentifier());
+    }
+    return results;
+  }
+
+  public void update(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                     String newMetadataLocation) throws SQLException {
+    queryRunner.update("UPDATE " + TABLES_TABLE_NAME + " SET metadataLocation = ? , previousMetadataLocation = ? " +
+                    "WHERE catalogName = ? AND tableNamespace = ? " +
+                    "AND tableName = ? ", newMetadataLocation,
+            oldMetadataLocation, catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());

Review comment:
       Looks like this doesn't yet implement an atomic swap of the old metadata location for the new one, so table updates would be unsafe because concurrent writers would clobber each other's commits. @kbendick, did your solution tackle that problem yet?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547561295



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {

Review comment:
       We would normally omit the "is" from a method name if it is clear that the return value is a boolean. I think that `exists` meets that requirement because it is natural to say "if table exists" instead of "if table is exists".




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r548237430



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {

Review comment:
       good idea implement it, and reorganized code a bit 
   >  I think an improvement would be to instantiate one per catalog instead (and pass in a connection pool so it can be shared with TableOperations). You'd probably want to rename it to something like TableSQL?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537767705



##########
File path: build.gradle
##########
@@ -231,6 +231,7 @@ project(':iceberg-core') {
       exclude group: 'org.slf4j', module: 'slf4j-log4j12'
     }
 
+    testCompile "com.h2database:h2"

Review comment:
       updated, wasn't sure about license compatibility, H2 license looked better.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537046259



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public JdbcTable get(TableIdentifier tableIdentifier) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ",

Review comment:
       moved statements to constants




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547572952



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {

Review comment:
       I'm not sure that I think this class is valuable. It is nice that it keeps track of the SQL statements and exposes some high-level methods, like `update`. The problem is that this provides many of the same operations as catalog, just under different names and with a slightly different API. For example, `getAll(Namespace)` is basically the same thing as `Catalog.listTables`.
   
   This class is also used in inconsistent ways. In `Catalog.delete` and in `TableOperations`, it is used for a single table even though it can work with any table and has no state other than catalog and connection. It is also used in `Catalog.listTables` and `Catalog.renameTable`, even though there is no table to encapsulate.
   
   I think that the problem is that this is attempting to encapsulate a table, but it is really more about abstracting some of the SQL details. I think an improvement would be to instantiate one per catalog instead (and pass in a connection pool so it can be shared with `TableOperations`). You'd probably want to rename it to something like `TableSQL`?




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


[GitHub] [iceberg] kbendick commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
kbendick commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-739050079


   Thanks for taking this up @ismailsimsek. I was going to push a PR for this either over the weekend or next Monday (have been really busy at work recently but I've got it working in spark-shell etc), but seems like you've beat me to it! I'll take a look through this one.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549528223



##########
File path: core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.Tasks;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS;
+import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS;
+import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestJdbcTableConcurrency {
+
+  static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("db", "test_table");
+  static final Schema SCHEMA = new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get())
+  );
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+  File tableDir;
+
+  @Test
+  public synchronized void testConcurrentFastAppends() throws IOException {
+    Map<String, String> properties = new HashMap<>();
+    this.tableDir = temp.newFolder();
+    properties.put(CatalogProperties.WAREHOUSE_LOCATION, tableDir.getAbsolutePath());
+    properties.put(CatalogProperties.HIVE_URI, "jdbc:h2:mem:concurentFastAppend;create=true");
+    JdbcCatalog catalog = new JdbcCatalog();
+    catalog.setConf(new Configuration());
+    catalog.initialize("jdbc", properties);
+    catalog.createTable(TABLE_IDENTIFIER, SCHEMA);
+
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+
+    String fileName = UUID.randomUUID().toString();
+    DataFile file = DataFiles.builder(icebergTable.spec())
+            .withPath(FileFormat.PARQUET.addExtension(fileName))

Review comment:
       Nit: continuation indents should be 2 indents, which is 4 spaces, not 8.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637620039



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.";

Review comment:
       This is a long parameter prefix. Could we make it shorter, like just `jdbc.`?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r539240650



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +

Review comment:
       we could ignore if  'ALTER TABLE my_table ADD PRIMARY KEY (columnx);' fails. so for many dbs we can get PK created




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r535858033



##########
File path: build.gradle
##########
@@ -226,6 +226,10 @@ project(':iceberg-core') {
     compile "com.fasterxml.jackson.core:jackson-databind"
     compile "com.fasterxml.jackson.core:jackson-core"
     compile "com.github.ben-manes.caffeine:caffeine"
+    compile 'commons-dbutils:commons-dbutils:1.7'
+    compile "org.hsqldb:hsqldb:2.5.1"
+    compile "mysql:mysql-connector-java:8.0.21"
+    compile "org.postgresql:postgresql:42.2.18"

Review comment:
       if it's only for testing, [sqlite](https://mvnrepository.com/artifact/org.xerial/sqlite-jdbc) is probably the safest choice




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536277772



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&

Review comment:
       I think it is sufficient to do `Preconditions.checkNotNull`, is there any benefit for doing this complicated check?
   
   

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);

Review comment:
       I think I prefer `DriverManager`, because `getConnection(String url, Properties info)` is much more flexible. Many JDBC connector needs more than username and password, for example AWS RDS needs `verifyServerCertificate` and `useSSL`. 
   
   I think instead of individual config fields, JDBC catalog can expose a config prefix `jdbccatalog.property.`, and all configs under this prefix would be added to properties and initialize a connection. For example, user name and password would become configs `jdbccatalog.property.username`, `jdbccatalog.property.password`.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537047495



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);

Review comment:
       switched to `DriverManager` and `Properties`. creating  Properties using prefix `jdbccatalog.property.`
   




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637618037



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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);

Review comment:
       We should also mimic the error messages from `HiveCatalog`: https://github.com/apache/iceberg/blob/master/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java#L222




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r654811321



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Database Connection timeout");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Database Connection failed");
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException(e, "Database data truncation error");
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException(e, "Database warning");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to connect to database");

Review comment:
       changed to "Unknown failure"




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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549534280



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));

Review comment:
       I think that the result set and prepared statement are tied to the connection that was used to create the statement. That means that as long as this method is using either the statement or the result set, the connection should not be reused by another thread.
   
   To hold the connection, the whole query will need to go in the `run` block. Here's my version of this method with that change:
   
   ```java
     @Override
     public List<TableIdentifier> listTables(Namespace namespace) {
       if (!this.namespaceExists(namespace)) {
         throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
       }
   
       try {
         return dbConnPool.run(conn -> {
           List<TableIdentifier> results = Lists.newArrayList();
           PreparedStatement sql = conn.prepareStatement(SQL_SELECT_ALL);
   
           sql.setString(1, catalogName);
           sql.setString(2, JdbcUtil.namespaceToString(namespace));
   
           try (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);
       }
     }
   ```
   
   I've also separated out the `InterruptedException` and handled it just like in `HiveCatalog`. I think that should be done in all cases here as well.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536327970



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);
+    BasicDataSource dataSource = new BasicDataSource();
+    dataSource.setDriverClassName(properties.get(JDBC_CATALOG_JDBC_DRIVER));
+    dataSource.setUrl(properties.get(JDBC_CATALOG_DBURL));
+    dataSource.setUsername(properties.get(JDBC_CATALOG_USER));
+    dataSource.setPassword(properties.get(JDBC_CATALOG_PASSWORD));
+    this.queryRunner = new QueryRunner(dataSource);
+
+    try {
+      initializeCatalog();
+    } catch (SQLException throwables) {
+      throw new RuntimeIOException("Failed to initialize Jdbc Catalog!\n %s %s", throwables.getErrorCode(),
+              throwables.getMessage());

Review comment:
       Exceptions should not be discarded. Instead, this should create a new exception with the caught exception as a cause.
   
   Also, we're moving from `RuntimeIOException` to Java's `UncheckedIOException`. So please use that instead.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537344744



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "default_catalog";
+  public static final String JDBC_CATALOG_TABLE_NAME = "iceberg_catalog";

Review comment:
       this seems complicated, usually connection url contains database name predefined and dbsession connects to it at the beginning. it seems complicated to create new database and switch to it. at the moment database shuld be created in advance and provided with dburl.
   
   currently catalog name kept in a column in both tables, `iceberg_namespaces` table to store namespace info and `iceberg_tables` to store table data.  it shuld be possible to have multiple catalogs.
   
   




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537130543



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public JdbcTable get(TableIdentifier tableIdentifier) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ",
+            new BeanHandler<>(JdbcTable.class),
+            catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());
+  }
+
+  public List<TableIdentifier> getAll(Namespace namespace) throws SQLException {
+    List<JdbcTable> tables = queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ?",
+            new BeanListHandler<>(JdbcTable.class), catalogName, namespace.toString());
+    List<TableIdentifier> results = Lists.newArrayList();
+    for (JdbcTable table : tables) {
+      results.add(table.toTableIdentifier());
+    }
+    return results;
+  }
+
+  public void update(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                     String newMetadataLocation) throws SQLException {
+    queryRunner.update("UPDATE " + TABLES_TABLE_NAME + " SET metadataLocation = ? , previousMetadataLocation = ? " +
+                    "WHERE catalogName = ? AND tableNamespace = ? " +
+                    "AND tableName = ? ", newMetadataLocation,
+            oldMetadataLocation, catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());

Review comment:
       changed the update statement if update statement effects >0 rows it will succeed otherwise will throw `CommitFailedException` exception.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r550481471



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));

Review comment:
       thank you for the example and tip applied it to all cases




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549529086



##########
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;

Review comment:
       Style: in other places, we use different naming conventions:
   
   * `FileIO` instances are typically called `io`
   * `Configuration` instances are typically called `conf`
   * A pool would typically be a plural named for the pooled objects, like `connections`




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


[GitHub] [iceberg] XuQianJin-Stars commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-784140777


   hi @ismailsimsek What is this current progress?


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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537046502



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&

Review comment:
       simplified it, its checking for null and empty string now




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


[GitHub] [iceberg] ismailsimsek commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-839639218


   hi @jackye1995 @rdblue @rymurr its ready for review
   if nothing major seem wrong it could be useful to merge and followup with improvements, enable other developers 


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637617059



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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;
+

Review comment:
       Nit: unnecessary newline at the end of the method.




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r535427137



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "default_catalog";

Review comment:
       nit: Following the convention of hive and hadoop catalog, the default name should probably be "jdbc"

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "default_catalog";
+  public static final String JDBC_CATALOG_TABLE_NAME = "iceberg_catalog";

Review comment:
       How do we incorporate the database concept in all SQL databases? Looks like we are just creating this table in the default database. My though on this is we can have a database named `iceberg` (maybe configurable), and inside it there can be a `namesapce` table to store namespace info, and this table (maybe with a different name like `tables`) to store all table information. Thoughts?

##########
File path: build.gradle
##########
@@ -226,6 +226,10 @@ project(':iceberg-core') {
     compile "com.fasterxml.jackson.core:jackson-databind"
     compile "com.fasterxml.jackson.core:jackson-core"
     compile "com.github.ben-manes.caffeine:caffeine"
+    compile 'commons-dbutils:commons-dbutils:1.7'
+    compile "org.hsqldb:hsqldb:2.5.1"
+    compile "mysql:mysql-connector-java:8.0.21"
+    compile "org.postgresql:postgresql:42.2.18"

Review comment:
       1. I only see the hsqldb used for testing, what are the use of mysql and postgres?
   2. should mark as `testCompile` instead
   3. versions should go to `versions.props`
   4. have you checked if the license is okay for hsqldb?

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "default_catalog";
+  public static final String JDBC_CATALOG_TABLE_NAME = "iceberg_catalog";
+  public static final String JDB_CATALOG_TABLE_DDL = "CREATE TABLE IF NOT EXISTS " +
+          JDBC_CATALOG_TABLE_NAME + " ( " +
+          "catalogName VARCHAR(1255) NOT NULL," +
+          "tableNamespace VARCHAR(1255)," +
+          "tableName VARCHAR(1255) NOT NULL," +
+          "metadataLocation VARCHAR(66255)," +
+          "previousMetadataLocation VARCHAR(66255)," +
+          "PRIMARY KEY (catalogName,tableNamespace,tableName)  " +
+          ")";
+  public static final String JDBC_NAMESPACE_TABLE_LIST = "SELECT catalogName, tableNamespace, tableName, " +
+          "metadataLocation, previousMetadataLocation FROM " + JDBC_CATALOG_TABLE_NAME + " " +
+          "WHERE catalogName = ? AND tableNamespace = ?";
+  public static final String JDBC_TABLE_RENAME = "UPDATE " + JDBC_CATALOG_TABLE_NAME + " SET tableNamespace = ? , " +
+          "tableName = ? WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_DROP = "DELETE FROM " + JDBC_CATALOG_TABLE_NAME + " WHERE catalogName = ? " +
+          "AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_SELECT = "SELECT catalogName, tableNamespace, tableName, metadataLocation, " +

Review comment:
       nit: why not just select *?

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "default_catalog";
+  public static final String JDBC_CATALOG_TABLE_NAME = "iceberg_catalog";
+  public static final String JDB_CATALOG_TABLE_DDL = "CREATE TABLE IF NOT EXISTS " +
+          JDBC_CATALOG_TABLE_NAME + " ( " +
+          "catalogName VARCHAR(1255) NOT NULL," +
+          "tableNamespace VARCHAR(1255)," +
+          "tableName VARCHAR(1255) NOT NULL," +
+          "metadataLocation VARCHAR(66255)," +
+          "previousMetadataLocation VARCHAR(66255)," +
+          "PRIMARY KEY (catalogName,tableNamespace,tableName)  " +
+          ")";
+  public static final String JDBC_NAMESPACE_TABLE_LIST = "SELECT catalogName, tableNamespace, tableName, " +
+          "metadataLocation, previousMetadataLocation FROM " + JDBC_CATALOG_TABLE_NAME + " " +
+          "WHERE catalogName = ? AND tableNamespace = ?";
+  public static final String JDBC_TABLE_RENAME = "UPDATE " + JDBC_CATALOG_TABLE_NAME + " SET tableNamespace = ? , " +
+          "tableName = ? WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_DROP = "DELETE FROM " + JDBC_CATALOG_TABLE_NAME + " WHERE catalogName = ? " +
+          "AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_SELECT = "SELECT catalogName, tableNamespace, tableName, metadataLocation, " +
+          "previousMetadataLocation FROM " + JDBC_CATALOG_TABLE_NAME + " WHERE catalogName = ? " +
+          "AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_INSERT = "INSERT INTO " + JDBC_CATALOG_TABLE_NAME + " (catalogName, " +
+          "tableNamespace, tableName, metadataLocation, previousMetadataLocation) VALUES (?,?,?,?,?)";
+  public static final String JDBC_TABLE_UPDATE_METADATA = "UPDATE " + JDBC_CATALOG_TABLE_NAME + " " +
+          "SET metadataLocation = ? , previousMetadataLocation = ? WHERE catalogName = ? AND tableNamespace = ? " +
+          "AND tableName = ? ";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private final QueryRunner queryRunner = new QueryRunner();
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection jdbcConnection;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void createCatalogIfNotExists() throws SQLException {
+    queryRunner.execute(jdbcConnection, JDB_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc table '{}' to store iceberg catalog!", JDBC_CATALOG_TABLE_NAME);
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    String catalogJdbcDriver = properties.get(JDBC_CATALOG_JDBC_DRIVER);
+    String catalogDburl = properties.get(JDBC_CATALOG_DBURL);
+    String catalogUser = properties.get(JDBC_CATALOG_USER);
+    String catalogDbpassword = properties.get(JDBC_CATALOG_PASSWORD);
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    DbUtils.loadDriver(catalogJdbcDriver);
+    LOG.debug("Connecting to Jdbc database {}.", catalogDburl);
+    try {
+      jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);
+      createCatalogIfNotExists();
+    } catch (SQLException throwables) {
+      throw new RuntimeIOException("Failed to initialize Jdbc Catalog!\n %s %s", throwables.getErrorCode(),
+              throwables.getMessage());
+    }
+
+  }
+
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(jdbcConnection, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) {
+    String tableName = tableIdentifier.name();
+    StringBuilder sb = new StringBuilder();
+
+    sb.append(warehouseLocation).append('/');
+    for (String level : tableIdentifier.namespace().levels()) {
+      sb.append(level).append('/');
+    }
+    sb.append(tableName);
+
+    return sb.toString();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      int insertedRecords = 0;
+      insertedRecords = queryRunner.update(jdbcConnection, JDBC_TABLE_DROP, catalogName,

Review comment:
       nit: the variable should be `deletedRecords`

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {

Review comment:
       we should be able to implement `SupportsNamespaces`, do you plan to do that in another PR?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637402550



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/UncheckedSQLException.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.SQLException;
+
+public class UncheckedSQLException extends RuntimeException {
+
+  public UncheckedSQLException(String message, SQLException cause) {

Review comment:
       applied




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637619883



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("Cannot drop table: {}! table not found in the catalog.", identifier);
+      return false;
+    }
+
+    TableOperations ops = newTableOps(identifier);

Review comment:
       If you instantiate the table ops here, then the table will never be cleaned up because it was just removed from the DB above so its metadata is lost.
   
   Like the Hive implementation, you need to instantiate the table operations, then run the drop only if the metadata is non-null. Otherwise the logic looks correct to me.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r548244177



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcNamespace.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcNamespace {

Review comment:
       removed namespace table and some of namespace methods




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549526107



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.sql.SQLWarning;
+import java.util.Properties;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+
+public class JdbcClientPool extends ClientPool<Connection, SQLException> {
+
+  private final String dbUrl;
+  private final Properties dbProperties;
+
+  JdbcClientPool(String dbUrl, Properties props) {
+    this((Integer) props.getOrDefault("iceberg.jdbc.client-pool-size", 5), dbUrl, props);
+  }
+
+  public JdbcClientPool(int poolSize, String dbUrl, Properties props) {
+    super(poolSize, SQLException.class);
+    dbProperties = props;
+    this.dbUrl = dbUrl;
+  }
+
+  @Override
+  protected Connection newClient() {
+    try {
+      return DriverManager.getConnection(dbUrl, dbProperties);
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Connection failed!", e);
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException("Database connection warning!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to connect to database!", e);

Review comment:
       `DriverManager.getConnection` will [throw `SQLTimeoutException` or a generic `SQLException`](https://docs.oracle.com/javase/8/docs/api/java/sql/DriverManager.html#getConnection-java.lang.String-java.util.Properties-). Those are the only two exception classes that need to be handled. Considering there isn't much that can be done for either one, I think this should simply wrap any `SQLException` in `UncheckedSQLException` with a little context, like the connect URI.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549534855



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);

Review comment:
       Other places use `Namespace does not exist: %s`




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537605790



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {

Review comment:
       i would expect potential exceptions like 'when the database is down', 'not reachable' or 'table is not there'. otherwise the result shuld return = null 
   maybe its better to re-throw the exception.. otherwise catalog class will assume table is not exists and will continue.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r654811272



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Database Connection timeout");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Database Connection failed");
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException(e, "Database data truncation error");
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException(e, "Database warning");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to connect to database");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during commit");
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to update the table %s from catalog %s " +
+          "Maybe another process changed it", tableIdentifier, catalogName);
+    }
+
+  }
+
+  private void createTable(String newMetadataLocation) throws SQLException, InterruptedException {
+    int insertRecord = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_TABLE_SQL)) {
+        sql.setString(1, catalogName);
+        sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(3, tableIdentifier.name());
+        sql.setString(4, newMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (insertRecord == 1) {
+      LOG.debug("Successfully committed to new table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to create table %s catalog %s", tableIdentifier, catalogName);
+    }
+  }
+
+  private void validateMetadataLocation(Map<String, String> table, TableMetadata base) {
+    String catalogMetadataLocation = !table.isEmpty() ? table.get(JdbcUtil.METADATA_LOCATION) : null;
+    String baseMetadataLocation = base != null ? base.metadataFileLocation() : null;
+
+    if (!Objects.equals(baseMetadataLocation, catalogMetadataLocation)) {
+      throw new CommitFailedException(
+          "Cannot commit %s because base metadata location '%s' is not same as the current Catalog location '%s'",

Review comment:
       updated




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


[GitHub] [iceberg] ismailsimsek commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-864454887


   > @ismailsimsek, this is looking really close. There are a few more minor issues but otherwise I think this is ready to go. Thank you for getting this done!
   
   Thank you for reviewing it @rdblue , its ready for another look


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637617304



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);

Review comment:
       I think we should base logging on what's done in the `HiveCatalog`. That way it is consistent. Here's the log for drop table: https://github.com/apache/iceberg/blob/master/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java#L175
   
   That also doesn't log if no table was dropped because it doesn't exist. I think that's the right call because we don't want to overwhelm the log with what didn't happen. At most that information should be logged at debug.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641568485



##########
File path: build.gradle
##########
@@ -231,6 +231,7 @@ project(':iceberg-core') {
       exclude group: 'org.slf4j', module: 'slf4j-log4j12'
     }
 
+    testCompile "com.h2database:h2"

Review comment:
       switched to org.xerial:sqlite-jdbc




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


[GitHub] [iceberg] rdblue commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-765055907


   Sorry, I got pulled away just after starting a review. I'll be able to do more tomorrow.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r562299584



##########
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");
   ```




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


[GitHub] [iceberg] ismailsimsek commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-865666167


   Thanks all for the reviews


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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r557048129



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.HIVE_URI, "").isEmpty(),

Review comment:
       #2088 




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536659722



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);

Review comment:
       Thank you for the hint it makes sense, will change it.




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


[GitHub] [iceberg] rdblue merged pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870


   


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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r535731354



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "default_catalog";
+  public static final String JDBC_CATALOG_TABLE_NAME = "iceberg_catalog";
+  public static final String JDB_CATALOG_TABLE_DDL = "CREATE TABLE IF NOT EXISTS " +
+          JDBC_CATALOG_TABLE_NAME + " ( " +
+          "catalogName VARCHAR(1255) NOT NULL," +
+          "tableNamespace VARCHAR(1255)," +
+          "tableName VARCHAR(1255) NOT NULL," +
+          "metadataLocation VARCHAR(66255)," +
+          "previousMetadataLocation VARCHAR(66255)," +
+          "PRIMARY KEY (catalogName,tableNamespace,tableName)  " +
+          ")";
+  public static final String JDBC_NAMESPACE_TABLE_LIST = "SELECT catalogName, tableNamespace, tableName, " +
+          "metadataLocation, previousMetadataLocation FROM " + JDBC_CATALOG_TABLE_NAME + " " +
+          "WHERE catalogName = ? AND tableNamespace = ?";
+  public static final String JDBC_TABLE_RENAME = "UPDATE " + JDBC_CATALOG_TABLE_NAME + " SET tableNamespace = ? , " +
+          "tableName = ? WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_DROP = "DELETE FROM " + JDBC_CATALOG_TABLE_NAME + " WHERE catalogName = ? " +
+          "AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_SELECT = "SELECT catalogName, tableNamespace, tableName, metadataLocation, " +

Review comment:
       changed




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637616092



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);

Review comment:
       We may want to consider an `UncheckedInterruptedException` like `UncheckedIOException` and `UncheckedSQLException`.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547568115



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  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 name = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection dbConn;

Review comment:
       This connection is passed into the `TableOperations` when creating a table, which means that it is potentially shared across threads. I don't think that connections are thread-safe, so that is a problem. For Hive, we implemented a connection pool that you should be able to reuse by extending `org.apache.iceberg.hive.ClientPool`. We may want to move that class into core so it can be used by both.




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-764180033


   #2088 is merged, please rebase when you have time, thanks!


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549529567



##########
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:
       Is this required by the database or is it just a convention? I don't think we should check for multiple table names.




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-764180033


   #2088 is merged, please rebase when you have time, thanks!


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549530302



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());

Review comment:
       This will result in `toString` called on the result of `levels()`, which is a `String[]`. That isn't correct. I think you want to pass `SLASH.join(table.namespace().levels())` into this `join` instead.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r553438924



##########
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:
       `%` needs to be set in the value itself, cannot add to the query. maybe i misunderstood ? 
   
   but removed if else logic for the second parameter by adding filter to lambda with a explanation.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549534280



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));

Review comment:
       I think that the result set and prepared statement are tied to the connection that was used to create the statement. That means that as long as this method is using either the statement or the result set, the connection should not be reused by another thread.
   
   To hold the connection, the whole query will need to go in the `run` block. Here's my version of this method with that change:
   
   ```java
     @Override
     public List<TableIdentifier> listTables(Namespace namespace) {
       if (!this.namespaceExists(namespace)) {
         throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
       }
   
       try {
         return dbConnPool.run(conn -> {
           List<TableIdentifier> results = Lists.newArrayList();
           try (PreparedStatement sql = conn.prepareStatement(SQL_SELECT_ALL)) {
             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);
       }
     }
   ```
   
   I've also separated out the `InterruptedException` and handled it just like in `HiveCatalog`. I think that should be done in all cases here as well.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549529164



##########
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")

Review comment:
       I don't think we should suppress this. Instead, can you update the code to avoid it?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r562300011



##########
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("/$", "");
   ```




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r553421199



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

Review comment:
       right removed it, it was useful at the beginning for understanding the code. 




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


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

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536416307



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);

Review comment:
       I agree on using `DriverManager`. Modern JDBC implementations will automatically register themselves and make themselves findable via the appropriate `META-INF/services` file. And yes, many different JDBC implementations / ways that people have stood up their relational databases require a lot more information.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r548262176



##########
File path: build.gradle
##########
@@ -231,6 +231,7 @@ project(':iceberg-core') {
       exclude group: 'org.slf4j', module: 'slf4j-log4j12'
     }
 
+    testRuntime "com.h2database:h2"

Review comment:
       updated




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536659722



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);

Review comment:
       Thank you for the hint it makes sense, will change it?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549529955



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;

Review comment:
       Why doesn't this store the table name that was found? Does it rely on case insensitive SQL behavior later?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549534986



##########
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 +

Review comment:
       Can you name these SQL statements for the methods that they are used in? For example, `LOAD_TABLE_SQL`, `RENAME_TABLE_SQL`, etc.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549531018



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(namespace));
+      ResultSet rs = sql.executeQuery();
+
+      while (rs.next()) {
+        final TableIdentifier table = JdbcUtil.stringToTableIdentifier(
+                rs.getString("table_namespace"), rs.getString("table_name"));
+        results.add(table);
+      }
+      rs.close();
+      return results;
+
+    } catch (SQLException | InterruptedException e) {
+      LOG.error("Failed to list tables!", e);
+      return null;

Review comment:
       This is not a correct return. If `SQLException` was thrown, then this should throw `UncheckedSQLException` with context about the operation, like `Cannot list namespace: %s`.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637617541



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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);

Review comment:
       I think this should be `info`. And we should follow the conventions in the `HiveCatalog`: https://github.com/apache/iceberg/blob/master/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java#L216




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r540092534



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public JdbcTable get(TableIdentifier tableIdentifier) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ",
+            new BeanHandler<>(JdbcTable.class),
+            catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());
+  }
+
+  public List<TableIdentifier> getAll(Namespace namespace) throws SQLException {
+    List<JdbcTable> tables = queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ?",
+            new BeanListHandler<>(JdbcTable.class), catalogName, namespace.toString());
+    List<TableIdentifier> results = Lists.newArrayList();
+    for (JdbcTable table : tables) {
+      results.add(table.toTableIdentifier());
+    }
+    return results;
+  }
+
+  public void update(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                     String newMetadataLocation) throws SQLException {
+    queryRunner.update("UPDATE " + TABLES_TABLE_NAME + " SET metadataLocation = ? , previousMetadataLocation = ? " +
+                    "WHERE catalogName = ? AND tableNamespace = ? " +
+                    "AND tableName = ? ", newMetadataLocation,
+            oldMetadataLocation, catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());

Review comment:
       to strengthen it further we can user[ row level locking](https://stackoverflow.com/a/46995306/2679740) 'SELECT FOR UPDATE'
   if its not implemented JDBC Driver Throws `SQLFeatureNotSupportedException`. using this exception we could fallback to regular update statement.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641541908



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("Cannot drop table: {}! table not found in the catalog.", identifier);
+      return false;
+    }
+
+    TableOperations ops = newTableOps(identifier);

Review comment:
       moved it to begining and aligned with hive

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("Cannot drop table: {}! table not found in the catalog.", identifier);
+      return false;
+    }
+
+    TableOperations ops = newTableOps(identifier);

Review comment:
       moved it to beginning and aligned with hive




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


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

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r632036694



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,394 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } 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 initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating tables(if missing) to store iceberg catalog");

Review comment:
       Nit: Missing space before opening parens of `(if missing)`.




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


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

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r551429257



##########
File path: core/src/main/java/org/apache/iceberg/CatalogUtil.java
##########
@@ -44,8 +44,10 @@
   public static final String ICEBERG_CATALOG_TYPE = "type";
   public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop";
   public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive";
+  public static final String ICEBERG_CATALOG_TYPE_JDBC = "jdbc";
   public static final String ICEBERG_CATALOG_HIVE = "org.apache.iceberg.hive.HiveCatalog";
   public static final String ICEBERG_CATALOG_HADOOP = "org.apache.iceberg.hadoop.HadoopCatalog";
+  public static final String ICEBERG_CATALOG_JDBC = "org.apache.iceberg.hadoop.HadoopCatalog";

Review comment:
       Shouldn't this refer to the `JdbcCatalog` class not `HadoopCatalog`?




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


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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r562917299



##########
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:
       moved this section to `initialize` seems to its too small to be separate




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


[GitHub] [iceberg] rdblue commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-865367950


   Thanks, @ismailsimsek for working on this! I just merged it.
   
   And thanks to @jackye1995, @kbendick, and @massdosage for reviewing!


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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r654811082



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {

Review comment:
       updated




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536325472



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java
##########
@@ -56,16 +56,16 @@
 import org.apache.iceberg.relocated.com.google.common.collect.Sets;
 
 /**
- * HadoopCatalog provides a way to use table names like db.table to work with path-based tables under a common
- * location. It uses a specified directory under a specified filesystem as the warehouse directory, and organizes
- * multiple levels directories that mapped to the database, namespace and the table respectively. The HadoopCatalog
- * takes a location as the warehouse directory. When creating a table such as $db.$tbl, it creates $db/$tbl
- * directory under the warehouse directory, and put the table metadata into that directory.
- *
- * The HadoopCatalog now supports {@link org.apache.iceberg.catalog.Catalog#createTable},
- * {@link org.apache.iceberg.catalog.Catalog#dropTable}, the {@link org.apache.iceberg.catalog.Catalog#renameTable}
- * is not supported yet.
- *
+ * HadoopCatalog provides a way to use table names like db.table to work with path-based tables under a common location.
+ * It uses a specified directory under a specified filesystem as the warehouse directory, and organizes multiple levels
+ * directories that mapped to the database, namespace and the table respectively. The HadoopCatalog takes a location as
+ * the warehouse directory. When creating a table such as $db.$tbl, it creates $db/$tbl directory under the warehouse
+ * directory, and put the table metadata into that directory.
+ * <p>
+ * The HadoopCatalog now supports {@link org.apache.iceberg.catalog.Catalog#NoSuchNamespaceExceptioncreateTable}, {@link
+ * org.apache.iceberg.catalog.Catalog#dropTable}, the {@link org.apache.iceberg.catalog.Catalog#renameTable} is not
+ * supported yet.
+ * <p>

Review comment:
       These changes don't look related. Could you add them in a separate PR?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637618214



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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 {
+

Review comment:
       Nit: no need for a newline here.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547574529



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {

Review comment:
       One more thing: it would be helpful if all of the methods in this class that correspond to methods in `Catalog` were named the same way as the ones in `Catalog`. For example, `getAll` is really `listTables` and `updateTableName` is really `renameTable`. Using names consistently will make it easier for people to understand what this is doing.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549530644



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(namespace));
+      ResultSet rs = sql.executeQuery();

Review comment:
       Could this use `try-with-resources` instead of calling close later?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537080968



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcNamespaceDao.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcNamespaceDao {
+  public static final String NAMESPACES_TABLE_NAME = "iceberg_namespaces";
+  public static final String NAMESPACES_TABLE_DDL = "CREATE TABLE IF NOT EXISTS " +
+          JdbcNamespaceDao.NAMESPACES_TABLE_NAME + " ( " +
+          "catalogName VARCHAR(1255) NOT NULL," +
+          "namespace VARCHAR(1255) NOT NULL," +
+          "namespaceMetadata VARCHAR(32768)," +
+          "PRIMARY KEY (catalogName, namespace) " +
+          ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcNamespaceDao.class);
+  private final String catalogName;
+  private QueryRunner queryRunner;
+
+
+  public JdbcNamespaceDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public JdbcNamespace get(Namespace namespace) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + NAMESPACES_TABLE_NAME + " WHERE catalogName = " +
+                    "? AND namespace = ? ;",
+            new BeanHandler<>(JdbcNamespace.class), catalogName, namespace.toString());
+  }
+
+  public boolean isExists(Namespace namespace) {
+    try {
+      return this.get(namespace) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public List<JdbcNamespace> getAll() throws SQLException {

Review comment:
       right seems like this one is not needed, but kept it and added   `public List<Namespace> getChildren(Namespace namespace)`




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537080968



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcNamespaceDao.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcNamespaceDao {
+  public static final String NAMESPACES_TABLE_NAME = "iceberg_namespaces";
+  public static final String NAMESPACES_TABLE_DDL = "CREATE TABLE IF NOT EXISTS " +
+          JdbcNamespaceDao.NAMESPACES_TABLE_NAME + " ( " +
+          "catalogName VARCHAR(1255) NOT NULL," +
+          "namespace VARCHAR(1255) NOT NULL," +
+          "namespaceMetadata VARCHAR(32768)," +
+          "PRIMARY KEY (catalogName, namespace) " +
+          ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcNamespaceDao.class);
+  private final String catalogName;
+  private QueryRunner queryRunner;
+
+
+  public JdbcNamespaceDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public JdbcNamespace get(Namespace namespace) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + NAMESPACES_TABLE_NAME + " WHERE catalogName = " +
+                    "? AND namespace = ? ;",
+            new BeanHandler<>(JdbcNamespace.class), catalogName, namespace.toString());
+  }
+
+  public boolean isExists(Namespace namespace) {
+    try {
+      return this.get(namespace) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public List<JdbcNamespace> getAll() throws SQLException {

Review comment:
       right seems like this one (`getAll()`) is not needed, but kept it and added   `public List<Namespace> getChildren(Namespace namespace)`




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r548215881



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();

Review comment:
       it was useful to rollback namespace changes when renaming a table fails(moving a table between catalogs). now we can remove it.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547566009



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return !this.get(tableIdentifier).isEmpty();
+    } catch (SQLException | JsonProcessingException throwables) {
+      return false;
+    }
+  }
+
+  public Map<String, String> get(TableIdentifier tableIdentifier) throws SQLException, JsonProcessingException {
+    Map<String, String> table = Maps.newHashMap();
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+    sql.setString(3, tableIdentifier.name());
+    ResultSet rs = sql.executeQuery();
+    if (rs.next()) {
+      table.put("catalog_name", rs.getString("catalog_name"));
+      table.put("table_namespace", rs.getString("table_namespace"));
+      table.put("table_name", rs.getString("table_name"));
+      table.put("metadata_location", rs.getString("metadata_location"));
+      table.put("previous_metadata_location", rs.getString("previous_metadata_location"));
+    }
+    rs.close();
+    return table;
+  }
+
+  public List<TableIdentifier> getAll(Namespace namespace) throws SQLException, JsonProcessingException {
+    List<TableIdentifier> results = Lists.newArrayList();
+
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT_ALL);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(namespace));
+    ResultSet rs = sql.executeQuery();
+
+    while (rs.next()) {
+      final TableIdentifier table = JdbcUtil.stringToTableIdentifier(
+              rs.getString("table_namespace"), rs.getString("table_name"));
+      results.add(table);
+    }
+    rs.close();
+    return results;
+  }
+
+  public int updateMetadataLocation(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                                    String newMetadataLocation) throws SQLException, JsonProcessingException {
+    PreparedStatement sql = dbConn.prepareStatement(SQL_UPDATE_METADATA_LOCATION);
+    // UPDATE
+    sql.setString(1, newMetadataLocation);
+    sql.setString(2, oldMetadataLocation);
+    // WHERE
+    sql.setString(3, catalogName);
+    sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+    sql.setString(5, tableIdentifier.name());
+    sql.setString(6, oldMetadataLocation);
+    return sql.executeUpdate();
+  }
+
+  public int save(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                  String newMetadataLocation) throws SQLException, JsonProcessingException {
+
+    PreparedStatement sql = dbConn.prepareStatement(SQL_INSERT);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+    sql.setString(3, tableIdentifier.name());
+    sql.setString(4, newMetadataLocation);
+    sql.setString(5, oldMetadataLocation);
+    return sql.executeUpdate();
+
+  }
+
+  public int updateTableName(TableIdentifier from, TableIdentifier to) throws SQLException, JsonProcessingException {
+
+    PreparedStatement sql = dbConn.prepareStatement(SQL_UPDATE_TABLE_NAME);
+    sql.setString(1, JdbcUtil.namespaceToString(to.namespace()));
+    sql.setString(2, to.name());
+    sql.setString(3, catalogName);
+    sql.setString(4, JdbcUtil.namespaceToString(from.namespace()));
+    sql.setString(5, from.name());
+    return sql.executeUpdate();
+
+  }
+
+  public int delete(TableIdentifier identifier) throws SQLException, JsonProcessingException {
+

Review comment:
       Nit: no need for a blank line at the start of a method.




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


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

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536417073



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);
+    BasicDataSource dataSource = new BasicDataSource();
+    dataSource.setDriverClassName(properties.get(JDBC_CATALOG_JDBC_DRIVER));
+    dataSource.setUrl(properties.get(JDBC_CATALOG_DBURL));
+    dataSource.setUsername(properties.get(JDBC_CATALOG_USER));
+    dataSource.setPassword(properties.get(JDBC_CATALOG_PASSWORD));
+    this.queryRunner = new QueryRunner(dataSource);
+
+    try {
+      initializeCatalog();
+    } catch (SQLException throwables) {
+      throw new RuntimeIOException("Failed to initialize Jdbc Catalog!\n %s %s", throwables.getErrorCode(),
+              throwables.getMessage());
+    }
+
+  }
+
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(queryRunner, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) {
+    String tableName = tableIdentifier.name();
+    StringBuilder sb = new StringBuilder();
+
+    sb.append(warehouseLocation).append('/');
+    for (String level : tableIdentifier.namespace().levels()) {
+      sb.append(level).append('/');
+    }
+    sb.append(tableName);
+
+    return sb.toString();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      new JdbcTableDao(queryRunner, catalogName).delete(identifier);
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        FileSystem fs = Util.getFs(new Path(warehouseLocation), hadoopConf);
+        fs.delete(new Path(lastMetadata.location()), true /* recursive */);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | IOException e) {
+      LOG.error("Cannot complete drop table operation for {} due to unexpected exception {}!", identifier,
+              e.getMessage(), e);
+      throw new RuntimeIOException("Failed to drop table %s", identifier.toString());
+    }
+  }
+
+  public void logTables() {
+    try {
+      List<JdbcTable> tables = queryRunner.query("SELECT * FROM " + JdbcTableDao.TABLES_TABLE_NAME,
+              new BeanListHandler<>(JdbcTable.class));
+      if (tables.isEmpty()) {
+        LOG.info("No Table found!");
+      }
+      for (JdbcTable table : tables) {
+        LOG.info("Table:{} ,NS: {}", table.toTableIdentifier().toString(), table.getTableNamespace().toString());
+        LOG.warn("IS NULL:{} ", table.getTableNamespace() == null);
+      }
+    } catch (SQLException throwables) {
+      LOG.error("Failed to list tables!", throwables);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+
+    JdbcNamespaceDao nsDao = new JdbcNamespaceDao(queryRunner, catalogName);
+    if (nsDao.isExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace.toString());
+    }
+    JdbcTableDao tableDao = new JdbcTableDao(queryRunner, catalogName);
+    List<TableIdentifier> results = Lists.newArrayList();

Review comment:
       Is this List allocation necessary? It seems like it's discarded when results is reassigned from `tableDao.getAll(namespace)` below.

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);
+    BasicDataSource dataSource = new BasicDataSource();
+    dataSource.setDriverClassName(properties.get(JDBC_CATALOG_JDBC_DRIVER));
+    dataSource.setUrl(properties.get(JDBC_CATALOG_DBURL));
+    dataSource.setUsername(properties.get(JDBC_CATALOG_USER));
+    dataSource.setPassword(properties.get(JDBC_CATALOG_PASSWORD));
+    this.queryRunner = new QueryRunner(dataSource);
+
+    try {
+      initializeCatalog();
+    } catch (SQLException throwables) {
+      throw new RuntimeIOException("Failed to initialize Jdbc Catalog!\n %s %s", throwables.getErrorCode(),
+              throwables.getMessage());
+    }
+
+  }
+

Review comment:
       Nit: More unnecessary white space.

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcNamespace.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iceberg.catalog.Namespace;
+
+public class JdbcNamespace {
+  private String catalogName;
+  private String namespace;
+  private String namespaceMetadata;
+
+  public JdbcNamespace(Namespace namespace) {
+    this.namespace = namespace.toString();

Review comment:
       Would it make more sense to just store the `Namespace` directly?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547565476



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return !this.get(tableIdentifier).isEmpty();
+    } catch (SQLException | JsonProcessingException throwables) {
+      return false;
+    }
+  }
+
+  public Map<String, String> get(TableIdentifier tableIdentifier) throws SQLException, JsonProcessingException {
+    Map<String, String> table = Maps.newHashMap();
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));

Review comment:
       Rather than using JSON to encode the namespace, I suggest converting it to a String using `Joiner.on(".")`.
   
   There's a trade-off to simplifying the problem by doing it that way: both ```a.`b.c` ``` and `a.b.c` end up as `a.b.c`. That has two effects:
   1. Namespaces that "collide" like that are considered the same namespace
   2. When listing namespace `a` the result is `[b]` and not `[b.c, b]`
   
   I think that those are fine. For the collisions, I think it is rare for users to _want_ namespaces that can collide but are not the same. If someone creates the table ```a.`b.c`.table_name``` then I really doubt that everyone wants to escape `b.c` every time they type the identifier. Everyone is going to type `a.b.c.table_name` anyway, which would return incorrect results if both tables exist. So not allowing namespaces that conflict is actually a good thing.
   
   This is exactly what Iceberg does to look up columns in a table, too. All column names are converted to a lookup key by joining the parts with `.`. Columns that have different structures but identical names aren't allowed, and I think it is better that way.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537046366



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {

Review comment:
       added it




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r553460848



##########
File path: core/src/main/java/org/apache/iceberg/CatalogUtil.java
##########
@@ -44,8 +44,10 @@
   public static final String ICEBERG_CATALOG_TYPE = "type";
   public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop";
   public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive";
+  public static final String ICEBERG_CATALOG_TYPE_JDBC = "jdbc";
   public static final String ICEBERG_CATALOG_HIVE = "org.apache.iceberg.hive.HiveCatalog";
   public static final String ICEBERG_CATALOG_HADOOP = "org.apache.iceberg.hadoop.HadoopCatalog";
+  public static final String ICEBERG_CATALOG_JDBC = "org.apache.iceberg.hadoop.HadoopCatalog";

Review comment:
       corrected it




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


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

Posted by GitBox <gi...@apache.org>.
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.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r535731152



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "default_catalog";

Review comment:
       changed to "jdbc"




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536326037



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+

Review comment:
       Nit: please don't use multiple whitespace lines. One is sufficient.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637403676



##########
File path: spark3/src/test/java/org/apache/iceberg/actions/TestRemoveOrphanFilesAction3.java
##########
@@ -165,4 +165,64 @@ public void testSparkSessionCatalogHiveTable() throws Exception {
     spark.conf().unset("spark.sql.catalog.spark_catalog.type");
   }
 
+  @Test
+  public void testSparkCatalogNamedJdbcTable() throws Exception {
+    spark.conf().set("spark.sql.catalog.jdbc", "org.apache.iceberg.spark.SparkCatalog");
+    spark.conf().set("spark.sql.catalog.jdbc.type", "jdbc");
+    spark.conf().set("spark.sql.catalog.jdbc.warehouse", tableLocation);
+    spark.conf().set("spark.sql.catalog.jdbc.uri", "jdbc:sqlite:file::memory:?icsparkjdbctestcat;");
+    spark.conf().set("spark.sql.catalog.jdbc.connection.parameter.user", "testuser");
+    spark.conf().set("spark.sql.catalog.jdbc.connection.parameter.password", "testpassword");
+    SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("jdbc");
+
+    String[] database = {"default"};
+    Identifier id = Identifier.of(database, "table");
+    Map<String, String> options = Maps.newHashMap();
+    Transform[] transforms = {};
+    cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options);
+    SparkTable table = cat.loadTable(id);
+
+    spark.sql("INSERT INTO jdbc.default.table VALUES (1,1,1)");
+
+    String location = table.table().location().replaceFirst("file:", "");
+    new File(location + "/data/trashfile").createNewFile();
+
+    List<String> results = Actions.forTable(table.table()).removeOrphanFiles()
+        .olderThan(System.currentTimeMillis() + 1000).execute();
+    Assert.assertTrue("trash file should be removed",
+        results.contains("file:" + location + "/data/trashfile"));
+  }
+
+  @Test
+  public void testSparkSessionCatalogJdbcTable() throws Exception {
+    spark.conf().set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog");
+    spark.conf().set("spark.sql.catalog.spark_catalog.type", "jdbc");
+    spark.conf().set("spark.sql.catalog.spark_catalog.warehouse", tableLocation);
+    spark.conf().set("spark.sql.catalog.spark_catalog.uri", "jdbc:sqlite:file::memory:?icsparktestcat;");
+    spark.conf().set("spark.sql.catalog.spark_catalog.connection.parameter.user", "testuser");
+    spark.conf().set("spark.sql.catalog.spark_catalog.connection.parameter.password", "testpassword");
+    SparkSessionCatalog cat = (SparkSessionCatalog) spark.sessionState().catalogManager().v2SessionCatalog();
+
+    String[] database = {"default"};
+    Identifier id = Identifier.of(database, "sessioncattest");
+    Map<String, String> options = Maps.newHashMap();
+    Transform[] transforms = {};
+    cat.dropTable(id);
+    cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options);
+    SparkTable table = (SparkTable) cat.loadTable(id);
+
+    spark.sql("INSERT INTO default.sessioncattest VALUES (1,1,1)");
+
+    String location = table.table().location().replaceFirst("file:", "");
+    new File(location + "/data/trashfile").createNewFile();
+
+    List<String> results = Actions.forTable(table.table()).removeOrphanFiles()
+        .olderThan(System.currentTimeMillis() + 1000).execute();
+    Assert.assertTrue("trash file should be removed",
+        results.contains("file:" + location + "/data/trashfile"));
+    // reset spark_catalog to default

Review comment:
       after rolling back CatalogUtil change i believe jdbc related test are not necessary, so rolled back the changes, it was duplication of existing tests, probably we could add better spark-jdbc-catalog tests with new PR? what do you think? 
   
   moved `reset...xxx` section to `resetSparkSessionCatalog` using `@After`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549534776



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);

Review comment:
       This case should return false, not throw an exception. That's the contract of the method specified in Javadoc.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549176406



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  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 name = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection dbConn;
+
+  public JdbcCatalog() {
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  @Override
+  public void initialize(String name, Map<String, String> properties) throws UncheckedIOException {
+    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.name = 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) throws UncheckedIOException {
+    try {
+
+      LOG.debug("Connecting to Jdbc database {}.", properties.get(CatalogProperties.HIVE_URI));
+      Properties connectionProps = new Properties();
+      for (Map.Entry<String, String> prop : properties.entrySet()) {
+        if (prop.getKey().startsWith(JDBC_PARAM_PREFIX)) {
+          connectionProps.put(prop.getKey().substring(JDBC_PARAM_PREFIX.length()), prop.getValue());
+        }
+      }
+
+      dbConn = DriverManager.getConnection(properties.get(CatalogProperties.HIVE_URI), connectionProps);
+      initializeCatalogTables();
+
+    } catch (SQLException throwables) {
+      throw new UncheckedIOException("Failed to initialize Jdbc Catalog!", new IOException(throwables));

Review comment:
       created `UncheckedSQLException` extending `UncheckedIOException`, and captured some specific sql exceptions. not sure if i got it 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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ismailsimsek commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-745196049


   @rdblue @kbendick @jackye1995 its ready for review.  "Java CI / check" failure doesn't seems related.


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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537043874



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;

Review comment:
       moved to `org.apache.iceberg.jdbc`




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637402619



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,394 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout", e);

Review comment:
       changed to `Cannot initialize jdbc catalog: Query timed out` 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549531018



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(namespace));
+      ResultSet rs = sql.executeQuery();
+
+      while (rs.next()) {
+        final TableIdentifier table = JdbcUtil.stringToTableIdentifier(
+                rs.getString("table_namespace"), rs.getString("table_name"));
+        results.add(table);
+      }
+      rs.close();
+      return results;
+
+    } catch (SQLException | InterruptedException e) {
+      LOG.error("Failed to list tables!", e);
+      return null;

Review comment:
       This is not a correct return. If `SQLException` was thrown, then this should throw `UncheckedSQLException` with context about the operation, like `Cannot list tables in namespace: %s`.




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r557039855



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.HIVE_URI, "").isEmpty(),
+        "No connection url provided for jdbc catalog!");
+    Preconditions.checkArgument(!properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "").isEmpty(),

Review comment:
       warehouse location is stored in the next line, so it make sense to first do `this.warehouseLocation = properties.getOrDefault(CatalogProperties.WAREHOUSE_LOCATION, "").replaceAll("/$", "")`, and then do all the condition checks against `this.warehouseLocation`. 

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.HIVE_URI, "").isEmpty(),

Review comment:
       Since this is now used by more than Hive, it does not make sense to say `HIVE_URI` now. Let me update that in another PR so that you can rebase it.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    } catch (SQLException e) {
+      // unknown exception happened when getting table from catalog
+      throw new UncheckedSQLException(
+          String.format("Failed to get table %s from catalog %s", tableIdentifier, catalogName), e);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s!" +
+          " maybe another process deleted it!", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault("metadata_location", null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get("metadata_location"));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it!
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists! maybe another process created it!");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Database Connection failed!", e);
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException("Database data truncation error!", e);
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException("Database warning!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to connect to database!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during commit", e);
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to commit the table %s from catalog %s! " +
+          "Maybe another process changed it!", tableIdentifier, catalogName);
+    }
+
+  }
+
+  private void createTable(String newMetadataLocation) throws SQLException, InterruptedException {
+    int insertRecord = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_SQL)) {
+        sql.setString(1, catalogName);
+        sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(3, tableIdentifier.name());
+        sql.setString(4, newMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (insertRecord == 1) {
+      LOG.debug("Successfully committed to new table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to commit the table %s from catalog %s", tableIdentifier, catalogName);
+    }
+  }
+
+  private void validateMetadataLocation(Map<String, String> table, TableMetadata base) {
+    String catalogMetadataLocation = !table.isEmpty() ? table.get("metadata_location") : null;
+    String baseMetadataLocation = base != null ? base.metadataFileLocation() : null;
+
+    if (!Objects.equals(baseMetadataLocation, catalogMetadataLocation)) {
+      throw new CommitFailedException(
+          "Cannot commit %s because base metadata location '%s' is not same as the current Catalog location '%s'",
+          tableIdentifier, baseMetadataLocation, catalogMetadataLocation);
+    }
+  }
+
+  @Override
+  public FileIO io() {
+    return fileIO;
+  }
+
+  @Override
+  protected String tableName() {
+    return tableIdentifier.toString();
+  }
+
+  private Map<String, String> getTable() throws UncheckedSQLException, SQLException, InterruptedException {
+    return connections.run(conn -> {
+      Map<String, String> table = Maps.newHashMap();
+
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.LOAD_TABLE_SQL)) {
+        sql.setString(1, catalogName);
+        sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(3, tableIdentifier.name());
+        ResultSet rs = sql.executeQuery();
+
+        if (rs.next()) {
+          table.put("catalog_name", rs.getString("catalog_name"));

Review comment:
       these column names should all be static variables in JdbcUtil

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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 org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+public final class JdbcUtil {
+  protected static final String SQL_TABLE_NAME = "iceberg_tables";

Review comment:
       I think `CATALOG_TABLE_NAME` is a better name, in case we will introduce other tables in the future.
   Also as discussed above, I think we can use upper case for the table name.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.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;

Review comment:
       `catalogName` is already `jdbc` if not set, so only need to do a `if` statement.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.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, PostgreSQL is keeping lowercase

Review comment:
       I just verified, postgres can accept all upper letter table name in queries. The table name is case insensitve anyway in SQL standard. So I think it is better to only check for upper case table name. Also change `SQL_TABLE_NAME` value to upper case.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during refresh", e);
+    } catch (SQLException e) {
+      // unknown exception happened when getting table from catalog
+      throw new UncheckedSQLException(
+          String.format("Failed to get table %s from catalog %s", tableIdentifier, catalogName), e);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s!" +
+          " maybe another process deleted it!", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault("metadata_location", null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get("metadata_location"));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it!
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists! maybe another process created it!");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Database Connection failed!", e);
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException("Database data truncation error!", e);
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException("Database warning!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to connect to database!", e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted during commit", e);
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to commit the table %s from catalog %s! " +
+          "Maybe another process changed it!", tableIdentifier, catalogName);
+    }
+
+  }
+
+  private void createTable(String newMetadataLocation) throws SQLException, InterruptedException {
+    int insertRecord = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_SQL)) {
+        sql.setString(1, catalogName);
+        sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(3, tableIdentifier.name());
+        sql.setString(4, newMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (insertRecord == 1) {
+      LOG.debug("Successfully committed to new table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to commit the table %s from catalog %s", tableIdentifier, catalogName);

Review comment:
       should indicate this is a create table operation, maybe message can be `fail to create the table ...`. Same for the update table exception above.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.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, PostgreSQL is keeping lowercase
+
+    boolean exists = connections.run(conn -> {
+      boolean foundTable = false;
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tables = dbMeta.getTables(null, null, JdbcUtil.SQL_TABLE_NAME, null);
+      if (tables.next()) {
+        foundTable = true;
+      }
+      tables.close();
+      ResultSet tablesUpper = dbMeta.getTables(null, null, JdbcUtil.SQL_TABLE_NAME.toUpperCase(), null);
+      if (tablesUpper.next()) {
+        foundTable = true;
+      }
+      tablesUpper.close();
+      ResultSet tablesLower = dbMeta.getTables(null, null, JdbcUtil.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(JdbcUtil.SQL_CREATE_CATALOG_TABLE).execute());
+      LOG.debug("Created table {} to store iceberg tables!", JdbcUtil.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(defaultNamespaceLocation(table.namespace()), table.name());
+    }
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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;

Review comment:
       nit: space after `if` statement

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,403 @@
+/*
+ * 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.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, PostgreSQL is keeping lowercase
+
+    boolean exists = connections.run(conn -> {
+      boolean foundTable = false;
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tables = dbMeta.getTables(null, null, JdbcUtil.SQL_TABLE_NAME, null);
+      if (tables.next()) {
+        foundTable = true;
+      }
+      tables.close();
+      ResultSet tablesUpper = dbMeta.getTables(null, null, JdbcUtil.SQL_TABLE_NAME.toUpperCase(), null);
+      if (tablesUpper.next()) {
+        foundTable = true;
+      }
+      tablesUpper.close();
+      ResultSet tablesLower = dbMeta.getTables(null, null, JdbcUtil.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(JdbcUtil.SQL_CREATE_CATALOG_TABLE).execute());
+      LOG.debug("Created table {} to store iceberg tables!", JdbcUtil.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(defaultNamespaceLocation(table.namespace()), table.name());
+    }
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());

Review comment:
       nit: space after `if` statement




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637616292



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));

Review comment:
       Nit: in other log messages and errors, JDBC is in all caps. I think we should use `JDBC` in messages for consistency, rather than `Jdbc`.




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536419432



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;

Review comment:
       The hadoop `Configurable` should have nothing to do with the location of this `JdbcCatalog`. It is only used by `HadoopFileIO`. Btw, I just noticed this is in the core module. Should it instead be in its own module?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537080968



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcNamespaceDao.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcNamespaceDao {
+  public static final String NAMESPACES_TABLE_NAME = "iceberg_namespaces";
+  public static final String NAMESPACES_TABLE_DDL = "CREATE TABLE IF NOT EXISTS " +
+          JdbcNamespaceDao.NAMESPACES_TABLE_NAME + " ( " +
+          "catalogName VARCHAR(1255) NOT NULL," +
+          "namespace VARCHAR(1255) NOT NULL," +
+          "namespaceMetadata VARCHAR(32768)," +
+          "PRIMARY KEY (catalogName, namespace) " +
+          ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcNamespaceDao.class);
+  private final String catalogName;
+  private QueryRunner queryRunner;
+
+
+  public JdbcNamespaceDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public JdbcNamespace get(Namespace namespace) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + NAMESPACES_TABLE_NAME + " WHERE catalogName = " +
+                    "? AND namespace = ? ;",
+            new BeanHandler<>(JdbcNamespace.class), catalogName, namespace.toString());
+  }
+
+  public boolean isExists(Namespace namespace) {
+    try {
+      return this.get(namespace) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public List<JdbcNamespace> getAll() throws SQLException {

Review comment:
       right seems like this one is not needed, but kept it and added   `public List<Namespace> getAllChilds(Namespace namespace)`

##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableOperations.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.Objects;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.iceberg.BaseMetastoreTableOperations;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.FileIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private QueryRunner queryRunner;
+
+  protected JdbcTableOperations(QueryRunner queryRunner, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+  }
+
+  // The doRefresh method should provide implementation on how to get the metadata location
+  @Override
+  public void doRefresh() {
+    String metadataLocation = null;
+    JdbcTableDao tableDao = new JdbcTableDao(queryRunner, catalogName);
+    JdbcTable table = null;
+    try {
+      table = tableDao.get(tableIdentifier);
+    } catch (SQLException throwables) {
+      LOG.debug("Table not found: {} , {}!", tableIdentifier, throwables);

Review comment:
       cleaned the logic with comments. for this line exception is re-thrown as `RuntimeException`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547560663



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcNamespace.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcNamespace {

Review comment:
       This adds quite a bit of complexity to the JDBC implementation because it requires a separate table for namespace and stores both namespace name and metadata as JSON objects. I think that ensuring consistency between the two tables adds a lot of unnecessary complexity. What happens if a table is added to a namespace that as it is concurrently deleted?
   
   I think a much simpler implementation is to omit the namespace table and determine whether a namespace exists based on whether there are any tables in it. As long as we can guarantee atomic changes to the tables table, consistency problems go away.
   
   That's also a simpler way to start a JDBC implementation. Then we can add namespace metadata later in a way that doesn't have those problems.
   
   @ismailsimsek what do you think about removing this and implementing `namespaceExists` based just on the tables table?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637616365



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);

Review comment:
       While I appreciate enthusiasm, there's no need for extra characters like `!` in log messages.




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


[GitHub] [iceberg] ismailsimsek commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-865666167


   Thanks all for the reviews


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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537049047



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcNamespace.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iceberg.catalog.Namespace;
+
+public class JdbcNamespace {
+  private String catalogName;
+  private String namespace;
+  private String namespaceMetadata;
+
+  public JdbcNamespace(Namespace namespace) {
+    this.namespace = namespace.toString();
+  }
+
+  public String getCatalogName() {
+    return catalogName;
+  }
+
+  public void setCatalogName(String catalogName) {
+    this.catalogName = catalogName;
+  }
+
+  public String getNamespace() {
+    return namespace;
+  }
+
+  public void setNamespace(String namespace) {
+    this.namespace = namespace;
+  }
+
+  public String getNamespaceMetadata() {
+    return namespaceMetadata;
+  }
+
+  public void setNamespaceMetadata(String namespaceMetadata) {
+    this.namespaceMetadata = namespaceMetadata;
+  }
+
+  public Namespace toNamespace() {
+    return Namespace.of(namespaceMetadata);
+  }
+
+  public Map<String, String> toNamespaceMetadata() throws JsonProcessingException {

Review comment:
       relaying on `toString()` implementation didn't felt safe so for both `namespace metadata` and `Namespace` I used Json mapper to convert to string("json sting"), and from.
   
   [JdbcUtil.java](https://github.com/ismailsimsek/iceberg/blob/iceberg_jdbccatalog/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java)
   
   ex:
   ```
     public static String namespaceToString(Namespace namespace) throws JsonProcessingException {
       return JsonUtil.mapper().writer().writeValueAsString(namespace.levels());
     }
   ```




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547916089



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcNamespace.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcNamespace {

Review comment:
       makes sense to keep initial implementation small and simple




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547573793



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();

Review comment:
       Are these needed? I think that autocommit can be used since all of the operations should just require a single command.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r654811254



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Database Connection timeout");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Database Connection failed");
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException(e, "Database data truncation error");
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException(e, "Database warning");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to connect to database");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during commit");
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to update the table %s from catalog %s " +
+          "Maybe another process changed it", tableIdentifier, catalogName);
+    }
+
+  }
+
+  private void createTable(String newMetadataLocation) throws SQLException, InterruptedException {
+    int insertRecord = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_TABLE_SQL)) {
+        sql.setString(1, catalogName);
+        sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(3, tableIdentifier.name());
+        sql.setString(4, newMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (insertRecord == 1) {
+      LOG.debug("Successfully committed to new table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to create table %s catalog %s", tableIdentifier, catalogName);
+    }
+  }
+
+  private void validateMetadataLocation(Map<String, String> table, TableMetadata base) {
+    String catalogMetadataLocation = !table.isEmpty() ? table.get(JdbcUtil.METADATA_LOCATION) : null;

Review comment:
       updated




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


[GitHub] [iceberg] ismailsimsek commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-739191859


   > Thanks for taking this up @ismailsimsek. I was going to push a PR for this either over the weekend or next Monday (have been really busy at work recently but I've got it working in spark-shell etc), but seems like you've beat me to it! I'll take a look through this one.
   
   Is it possible combine best of both? I'm still learning it and I'm sure there is some stuff this implementain is still lacking.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547575231



##########
File path: core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java
##########
@@ -0,0 +1,566 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.Transaction;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.transforms.Transform;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.NullOrder.NULLS_FIRST;
+import static org.apache.iceberg.SortDirection.ASC;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestJdbcCatalog {
+  // Schema passed to create tables
+  static final Schema SCHEMA = new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get())
+  );
+
+  // This is the actual schema for the table, with column IDs reassigned
+  static final Schema TABLE_SCHEMA = new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get())
+  );
+
+  // Partition spec used to create tables
+  static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .bucket("data", 16)
+          .build();
+
+  private static final ImmutableMap<String, String> meta = ImmutableMap.of();
+  static Configuration conf;
+  private static JdbcCatalog catalog;
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+  File tableDir = null;
+  String tableLocation = null;
+  File versionHintFile = null;
+  String warehousePath;
+
+  protected List<String> metadataVersionFiles(String location) {
+    return Stream.of(new File(location).listFiles())
+            .filter(file -> !file.isDirectory())
+            .map(File::getName)
+            .filter(fileName -> fileName.endsWith("metadata.json"))
+            .collect(Collectors.toList())
+            ;
+  }
+
+  protected List<String> manifestFiles(String location) {
+    return Stream.of(new File(location).listFiles())
+            .filter(file -> !file.isDirectory())
+            .map(File::getName)
+            .filter(fileName -> fileName.endsWith(".avro"))
+            .collect(Collectors.toList())
+            ;
+  }
+
+  @Before
+  public void setupTable() throws Exception {
+    this.tableDir = temp.newFolder();
+    tableDir.delete(); // created by table create
+
+    this.tableLocation = tableDir.toURI().toString();
+    this.versionHintFile = new File(new File(tableDir, "metadata"), "version-hint.text");

Review comment:
       This shouldn't be needed because only Hadoop tables use version hints.




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-813644518


   Sorry I think we forgot to move forward with this PR, I will take a look after the tests passed.


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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r550481986



##########
File path: core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.Tasks;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS;
+import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS;
+import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestJdbcTableConcurrency {
+
+  static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("db", "test_table");
+  static final Schema SCHEMA = new Schema(
+          required(1, "id", Types.IntegerType.get(), "unique ID"),
+          required(2, "data", Types.StringType.get())
+  );
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+  File tableDir;
+
+  @Test
+  public synchronized void testConcurrentFastAppends() throws IOException {
+    Map<String, String> properties = new HashMap<>();
+    this.tableDir = temp.newFolder();
+    properties.put(CatalogProperties.WAREHOUSE_LOCATION, tableDir.getAbsolutePath());
+    properties.put(CatalogProperties.HIVE_URI, "jdbc:h2:mem:concurentFastAppend;create=true");
+    JdbcCatalog catalog = new JdbcCatalog();
+    catalog.setConf(new Configuration());
+    catalog.initialize("jdbc", properties);
+    catalog.createTable(TABLE_IDENTIFIER, SCHEMA);
+
+    Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER);
+
+    String fileName = UUID.randomUUID().toString();
+    DataFile file = DataFiles.builder(icebergTable.spec())
+            .withPath(FileFormat.PARQUET.addExtension(fileName))

Review comment:
       updated my editor setting and reformatted files.




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


[GitHub] [iceberg] ismailsimsek edited a comment on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek edited a comment on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-745196049


   @rdblue @kbendick @jackye1995 its ready for review.  "Java CI / check" failure doesn't seem related.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549530586



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {

Review comment:
       Style: please add newlines after control flow blocks like `if`, `for`, etc.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641542736



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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(JdbcUtil.LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.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
+          .filter(n -> n.levels().length >= subNamespaceLevelLength)
+          // 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()) {

Review comment:
       updated, now empty namespaces are allowed




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r655724859



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);

Review comment:
       Nice job catching and fixing that.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637617981



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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");

Review comment:
       This violates our assumption about tables, but it doesn't mean the operation failed. Javadoc states that this will return the number of affected rows or 0. So if this isn't 0 or 1, then it means that more than 1 row was affected and our underlying primary key assumption is violated. I think in that case we want an error message that highlights the problem at WARNING: "Rename operation affected %s rows: the catalog table's primary key assumption has been violated"




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637619716



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));

Review comment:
       Minor: it is more readable to wrap lines at the top-level function call's arguments rather than in the middle of those arguments.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549528014



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.sql.SQLWarning;
+import java.util.Properties;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+
+public class JdbcClientPool extends ClientPool<Connection, SQLException> {
+
+  private final String dbUrl;
+  private final Properties dbProperties;
+
+  JdbcClientPool(String dbUrl, Properties props) {
+    this((Integer) props.getOrDefault("iceberg.jdbc.client-pool-size", 5), dbUrl, props);
+  }
+
+  public JdbcClientPool(int poolSize, String dbUrl, Properties props) {
+    super(poolSize, SQLException.class);
+    dbProperties = props;
+    this.dbUrl = dbUrl;
+  }
+
+  @Override
+  protected Connection newClient() {
+    try {
+      return DriverManager.getConnection(dbUrl, dbProperties);
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Connection timeout!", e);
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException("Connection failed!", e);
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException("Database connection warning!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to connect to database!", e);
+    }
+  }
+
+  @Override
+  protected Connection reconnect(Connection client) {
+    this.close(client);

Review comment:
       No need to prefix with `this` when calling a method or reading a field. We only add it to distinguish between setting instance fields (`this.x = ...`) and local variables (`x = ...`).




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


[GitHub] [iceberg] rdblue commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-765055907


   Sorry, I got pulled away just after starting a review. I'll be able to do more tomorrow.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549534584



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(namespace));
+      ResultSet rs = sql.executeQuery();

Review comment:
       I fixed this in the version above. That uses a try-with-resources block for the prepared statement so that all of the resources are released before releasing the connection back to the pool.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549524804



##########
File path: api/src/main/java/org/apache/iceberg/exceptions/UncheckedSQLException.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.exceptions;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.SQLException;
+
+public class UncheckedSQLException extends UncheckedIOException {

Review comment:
       Why extend `UncheckedIOException`? This class is supposed to be the equivalent for `SQLException`, which is not an `IOException`.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641567328



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);

Review comment:
       replaced all with `UncheckedInterruptedException`




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r550468895



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;

Review comment:
       this one checking if the table exists and creates it if it is not. equivalent of `CREATE TABLE IF NOT EXISTS` since MS sql server not supporting `IF NOT EXISTS` created this method to do the check(if the catalog table already created or not).
   
   `.getTables` is doing case sensitive table check. and some databases are keeping table names uppercase that's why its doing multiple checks(lowercase, uppercase, as is). same [question is here ](https://stackoverflow.com/a/53061323/2679740)
   
   its not elegant but seems like working, few options i think of are
   
   1. use `CREATE TABLE IF NOT EXISTS` 
   2. fetch all tables once with `dbMeta.getTables` and then look for the table with case insensitive match




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r546029484



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+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.hadoop.Util;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+
+  public static final String JDBC_PARAM_PREFIX = "connection.parameter.";
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String TABLE_METADATA_FILE_EXTENSION = ".metadata.json";
+  private static final PathFilter TABLE_FILTER = path -> path.getName().endsWith(TABLE_METADATA_FILE_EXTENSION);
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private FileIO fileIO;
+  private FileSystem fs;

Review comment:
       you are right, removed it




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r535731000



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "default_catalog";
+  public static final String JDBC_CATALOG_TABLE_NAME = "iceberg_catalog";
+  public static final String JDB_CATALOG_TABLE_DDL = "CREATE TABLE IF NOT EXISTS " +
+          JDBC_CATALOG_TABLE_NAME + " ( " +
+          "catalogName VARCHAR(1255) NOT NULL," +
+          "tableNamespace VARCHAR(1255)," +
+          "tableName VARCHAR(1255) NOT NULL," +
+          "metadataLocation VARCHAR(66255)," +
+          "previousMetadataLocation VARCHAR(66255)," +
+          "PRIMARY KEY (catalogName,tableNamespace,tableName)  " +
+          ")";
+  public static final String JDBC_NAMESPACE_TABLE_LIST = "SELECT catalogName, tableNamespace, tableName, " +
+          "metadataLocation, previousMetadataLocation FROM " + JDBC_CATALOG_TABLE_NAME + " " +
+          "WHERE catalogName = ? AND tableNamespace = ?";
+  public static final String JDBC_TABLE_RENAME = "UPDATE " + JDBC_CATALOG_TABLE_NAME + " SET tableNamespace = ? , " +
+          "tableName = ? WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_DROP = "DELETE FROM " + JDBC_CATALOG_TABLE_NAME + " WHERE catalogName = ? " +
+          "AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_SELECT = "SELECT catalogName, tableNamespace, tableName, metadataLocation, " +
+          "previousMetadataLocation FROM " + JDBC_CATALOG_TABLE_NAME + " WHERE catalogName = ? " +
+          "AND tableNamespace = ? AND tableName = ? ";
+  public static final String JDBC_TABLE_INSERT = "INSERT INTO " + JDBC_CATALOG_TABLE_NAME + " (catalogName, " +
+          "tableNamespace, tableName, metadataLocation, previousMetadataLocation) VALUES (?,?,?,?,?)";
+  public static final String JDBC_TABLE_UPDATE_METADATA = "UPDATE " + JDBC_CATALOG_TABLE_NAME + " " +
+          "SET metadataLocation = ? , previousMetadataLocation = ? WHERE catalogName = ? AND tableNamespace = ? " +
+          "AND tableName = ? ";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private final QueryRunner queryRunner = new QueryRunner();
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection jdbcConnection;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void createCatalogIfNotExists() throws SQLException {
+    queryRunner.execute(jdbcConnection, JDB_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc table '{}' to store iceberg catalog!", JDBC_CATALOG_TABLE_NAME);
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    String catalogJdbcDriver = properties.get(JDBC_CATALOG_JDBC_DRIVER);
+    String catalogDburl = properties.get(JDBC_CATALOG_DBURL);
+    String catalogUser = properties.get(JDBC_CATALOG_USER);
+    String catalogDbpassword = properties.get(JDBC_CATALOG_PASSWORD);
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    DbUtils.loadDriver(catalogJdbcDriver);
+    LOG.debug("Connecting to Jdbc database {}.", catalogDburl);
+    try {
+      jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);
+      createCatalogIfNotExists();
+    } catch (SQLException throwables) {
+      throw new RuntimeIOException("Failed to initialize Jdbc Catalog!\n %s %s", throwables.getErrorCode(),
+              throwables.getMessage());
+    }
+
+  }
+
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(jdbcConnection, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) {
+    String tableName = tableIdentifier.name();
+    StringBuilder sb = new StringBuilder();
+
+    sb.append(warehouseLocation).append('/');
+    for (String level : tableIdentifier.namespace().levels()) {
+      sb.append(level).append('/');
+    }
+    sb.append(tableName);
+
+    return sb.toString();
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      int insertedRecords = 0;
+      insertedRecords = queryRunner.update(jdbcConnection, JDBC_TABLE_DROP, catalogName,

Review comment:
       updated




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536896606



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {
+      return false;
+    }
+  }
+
+  public JdbcTable get(TableIdentifier tableIdentifier) throws SQLException {
+    return queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ? AND tableName = ? ",
+            new BeanHandler<>(JdbcTable.class),
+            catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());
+  }
+
+  public List<TableIdentifier> getAll(Namespace namespace) throws SQLException {
+    List<JdbcTable> tables = queryRunner.query("SELECT * FROM " + TABLES_TABLE_NAME + " " +
+                    "WHERE catalogName = ? AND tableNamespace = ?",
+            new BeanListHandler<>(JdbcTable.class), catalogName, namespace.toString());
+    List<TableIdentifier> results = Lists.newArrayList();
+    for (JdbcTable table : tables) {
+      results.add(table.toTableIdentifier());
+    }
+    return results;
+  }
+
+  public void update(TableIdentifier tableIdentifier, String oldMetadataLocation,
+                     String newMetadataLocation) throws SQLException {
+    queryRunner.update("UPDATE " + TABLES_TABLE_NAME + " SET metadataLocation = ? , previousMetadataLocation = ? " +
+                    "WHERE catalogName = ? AND tableNamespace = ? " +
+                    "AND tableName = ? ", newMetadataLocation,
+            oldMetadataLocation, catalogName, tableIdentifier.namespace().toString(), tableIdentifier.name());

Review comment:
       I think all you would need to do is to change the SQL slightly:
   
   ```sql
   UPDATE iceberg_tables
   SET metadata_location = ?, previous_metadata_location = ?
   WHERE table_namespace = ? AND table_name = ? AND metadata_location = ?
   ```
   
   That adds a predicate to ensure that the table namespace and name combination will only be changed if the metadata_location is the expected one. If that affects 0 rows, then the metadata location was changed concurrently. If it affects 1 row, the commit was successful, and if it affects more than one row, there's a problem. (Note that I've simplified this to omit `catalog_name`.)




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r539512479



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +

Review comment:
       I think the problem is that we need that primary key to ensure that there is only one entry for a given table identifier. We don't want two concurrent creates to collide and create two different tables.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r546031060



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+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.hadoop.Util;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+
+  public static final String JDBC_PARAM_PREFIX = "connection.parameter.";
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String TABLE_METADATA_FILE_EXTENSION = ".metadata.json";
+  private static final PathFilter TABLE_FILTER = path -> path.getName().endsWith(TABLE_METADATA_FILE_EXTENSION);
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private FileIO fileIO;
+  private FileSystem fs;
+  private String name = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection dbConn;
+
+  public JdbcCatalog(String name, Configuration conf, String warehouseLocation, Map<String, String> properties) {

Review comment:
       moved the logic to `initialize` 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547569028



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  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 name = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection dbConn;
+
+  public JdbcCatalog() {
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  @Override
+  public void initialize(String name, Map<String, String> properties) throws UncheckedIOException {
+    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.name = 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) throws UncheckedIOException {
+    try {
+
+      LOG.debug("Connecting to Jdbc database {}.", properties.get(CatalogProperties.HIVE_URI));
+      Properties connectionProps = new Properties();
+      for (Map.Entry<String, String> prop : properties.entrySet()) {
+        if (prop.getKey().startsWith(JDBC_PARAM_PREFIX)) {
+          connectionProps.put(prop.getKey().substring(JDBC_PARAM_PREFIX.length()), prop.getValue());
+        }
+      }
+
+      dbConn = DriverManager.getConnection(properties.get(CatalogProperties.HIVE_URI), connectionProps);
+      initializeCatalogTables();
+
+    } catch (SQLException throwables) {
+      throw new UncheckedIOException("Failed to initialize Jdbc Catalog!", new IOException(throwables));

Review comment:
       Also, if we can catch more specific subclasses, that would be ideal. Here's a list of them: https://docs.oracle.com/javase/7/docs/api/java/sql/SQLException.html




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536894358



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);

Review comment:
       Minor: Rather than breaking in the middle of a method call, the ternary operator provides good places to break lines:
   
   ```java
   this.fileIO = fileIOImpl == null ?
      new HadoopFileIO(hadoopConf) :
      CatalogUtil.loadFileIO(fileIOImpl, properties, hadoopConf);
   ```




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r654811816



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);

Review comment:
       it was stuck in infinite loop so i had to call disableRefresh(); before the return
   
   lopp is doing 
   ```
   	at org.apache.iceberg.BaseMetastoreTableOperations.current(BaseMetastoreTableOperations.java:77)
   	at org.apache.iceberg.BaseMetastoreTableOperations.refresh(BaseMetastoreTableOperations.java:106)
   ```
   
   ```
   
       if (table.isEmpty()) {
         if (currentMetadataLocation() != null) {
           throw new NoSuchTableException("Failed to load table %s from catalog %s: dropped by another process",
               tableIdentifier, catalogName);
         } else {
           this.disableRefresh(); # new method added to `BaseMetastoreTableOperations` sets shouldRefresh=false
           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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r654811987



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");

Review comment:
       updated with following if condition
   
   ```
         if (currentMetadataLocation() == null) {
           throw new AlreadyExistsException(e, "Table already exists: %s", tableIdentifier);
         } else {
           throw new UncheckedSQLException(e, "Table already exists: %s", tableIdentifier);
         }
   ```




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r540088315



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +

Review comment:
       using [sqlfiddle](http://sqlfiddle.com/) for testing all databases there are (mysql, MS sql, postgresql, oracle )
   - compatible having `PRIMARY KEY` in the table DDL. 
   - MS SQL Server not supporting create Table `IF NOT EXISTS`




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641501787



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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(JdbcUtil.LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.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
+          .filter(n -> n.levels().length >= subNamespaceLevelLength)
+          // 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);
+    }
+
+    return ImmutableMap.of("location", defaultNamespaceLocation(namespace));
+  }
+
+  private String defaultNamespaceLocation(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return warehouseLocation;
+    } else {
+      return SLASH.join(warehouseLocation, SLASH.join(namespace.levels()));
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Cannot drop namespace %s because it is not found!", namespace);
+    }
+
+    List<TableIdentifier> tableIdentifiers = listTables(namespace);
+    if (tableIdentifiers != null && !tableIdentifiers.isEmpty()) {
+      throw new NamespaceNotEmptyException("Cannot drop namespace %s because it is not empty. " +

Review comment:
       aligned log message with hive




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r641541654



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);

Review comment:
       aligned logs with hive, 
   
   it seems when table not exists hive [throws exception](https://github.com/apache/iceberg/blob/15616941461aaad3f03568ce4880171b86a06547/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java#L178) then its handled in catch section, on jdbc case there is no exception thrown but deletedRecords=0 , what do you think changing loglevel to debug and keeping `retun false` ?
   
   
   
   ```
   } catch (NoSuchTableException | NoSuchObjectException e) {
   LOG.info("Skipping drop, table does not exist: {}", identifier, e);
   <span class="pl-k">return</span> <span class="pl-c1">false</span>;
   ```




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r654051672



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);

Review comment:
       I'm okay with either debug or info for this one.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Database Connection timeout");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Database Connection failed");
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException(e, "Database data truncation error");
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException(e, "Database warning");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to connect to database");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during commit");
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to update the table %s from catalog %s " +
+          "Maybe another process changed it", tableIdentifier, catalogName);
+    }
+
+  }
+
+  private void createTable(String newMetadataLocation) throws SQLException, InterruptedException {
+    int insertRecord = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_TABLE_SQL)) {
+        sql.setString(1, catalogName);
+        sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(3, tableIdentifier.name());
+        sql.setString(4, newMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (insertRecord == 1) {
+      LOG.debug("Successfully committed to new table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to create table %s catalog %s", tableIdentifier, catalogName);

Review comment:
       Nit: should be "in catalog" not just "catalog"

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Database Connection timeout");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Database Connection failed");
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException(e, "Database data truncation error");
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException(e, "Database warning");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to connect to database");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during commit");
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to update the table %s from catalog %s " +
+          "Maybe another process changed it", tableIdentifier, catalogName);

Review comment:
       Nit: No need for "maybe" here. It is enough to say that the table has changed.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);

Review comment:
       Similar to below, no need for the "maybe" statement. This failed to load a table that it expected to exist.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);

Review comment:
       Because `currentMetadataLocation()` must be `null` because of the check above, this will do basically nothing. I see that this call is probably here because Hive always calls `refreshFromMetadataLocation`, but I don't think that it is necessary to call it here.
   
   I'd probably combine this with the previous check:
   
   ```java
   if (table.isEmpty()) {
     if (currentMetadataLocation() != null) {
       throw new NoSuchTableException("Failed to load table %s from catalog %s: dropped by another process", ...);
     } else {
       return;
     }
   }
   ```

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {

Review comment:
       This `get` is done twice. Can you refactor to use a variable? I think this should also be `IllegalStateException` rather than just a `RuntimeException` so you could do this:
   
   ```java
   String newMetadataLocation = table.get(JdbcUtil.METADATA_LOCATION);
   Preconditions.checkState(newMetadataLocation != null, "Invalid table %s: metadata location is null", tableIdentifier);
   
   refreshFromMetadataLocation(newMetadataLocation);
   ```

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {

Review comment:
       (Also, `getOrDefault` with a `null` default is equivalent to `get`.)

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Database Connection timeout");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Database Connection failed");
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException(e, "Database data truncation error");
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException(e, "Database warning");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to connect to database");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during commit");
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to update the table %s from catalog %s " +
+          "Maybe another process changed it", tableIdentifier, catalogName);
+    }
+
+  }
+
+  private void createTable(String newMetadataLocation) throws SQLException, InterruptedException {
+    int insertRecord = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_TABLE_SQL)) {
+        sql.setString(1, catalogName);
+        sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(3, tableIdentifier.name());
+        sql.setString(4, newMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (insertRecord == 1) {
+      LOG.debug("Successfully committed to new table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to create table %s catalog %s", tableIdentifier, catalogName);
+    }
+  }
+
+  private void validateMetadataLocation(Map<String, String> table, TableMetadata base) {
+    String catalogMetadataLocation = !table.isEmpty() ? table.get(JdbcUtil.METADATA_LOCATION) : null;

Review comment:
       I think this can just be `table.get(JdbcUtil.METADATA_LOCATION)`.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Database Connection timeout");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Database Connection failed");
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException(e, "Database data truncation error");
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException(e, "Database warning");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to connect to database");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during commit");
+    }
+  }
+
+  private void updateTable(String newMetadataLocation, String oldMetadataLocation)
+      throws SQLException, InterruptedException {
+    int updatedRecords = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) {
+        // UPDATE
+        sql.setString(1, newMetadataLocation);
+        sql.setString(2, oldMetadataLocation);
+        // WHERE
+        sql.setString(3, catalogName);
+        sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(5, tableIdentifier.name());
+        sql.setString(6, oldMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (updatedRecords == 1) {
+      LOG.debug("Successfully committed to existing table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to update the table %s from catalog %s " +
+          "Maybe another process changed it", tableIdentifier, catalogName);
+    }
+
+  }
+
+  private void createTable(String newMetadataLocation) throws SQLException, InterruptedException {
+    int insertRecord = connections.run(conn -> {
+      try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_TABLE_SQL)) {
+        sql.setString(1, catalogName);
+        sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));
+        sql.setString(3, tableIdentifier.name());
+        sql.setString(4, newMetadataLocation);
+        return sql.executeUpdate();
+      }
+    });
+
+    if (insertRecord == 1) {
+      LOG.debug("Successfully committed to new table: {}", tableIdentifier);
+    } else {
+      throw new CommitFailedException("Failed to create table %s catalog %s", tableIdentifier, catalogName);
+    }
+  }
+
+  private void validateMetadataLocation(Map<String, String> table, TableMetadata base) {
+    String catalogMetadataLocation = !table.isEmpty() ? table.get(JdbcUtil.METADATA_LOCATION) : null;
+    String baseMetadataLocation = base != null ? base.metadataFileLocation() : null;
+
+    if (!Objects.equals(baseMetadataLocation, catalogMetadataLocation)) {
+      throw new CommitFailedException(
+          "Cannot commit %s because base metadata location '%s' is not same as the current Catalog location '%s'",

Review comment:
       Nit: error message could be more concise: `"Cannot commit %s: metadata location %s has changed from %s"`

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");

Review comment:
       I think this should only be thrown if `currentMetadataLocation()` is `null` because that indicates this is a new table. Otherwise, I think this should throw `UncheckedSQLException`.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);
+      return;
+    }
+
+    // Table exists but metadataLocation is null
+    if (table.getOrDefault(JdbcUtil.METADATA_LOCATION, null) == null) {
+      throw new RuntimeException(String.format("Failed to get metadata location of the table %s from catalog %s",
+          tableIdentifier, catalogName));
+    }
+
+    refreshFromMetadataLocation(table.get(JdbcUtil.METADATA_LOCATION));
+  }
+
+  @Override
+  public void doCommit(TableMetadata base, TableMetadata metadata) {
+    String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1);
+    try {
+      Map<String, String> table = getTable();
+
+      if (!table.isEmpty()) {
+        validateMetadataLocation(table, base);
+        String oldMetadataLocation = base.metadataFileLocation();
+        // Start atomic update
+        LOG.debug("Committing existing table: {}", tableName());
+        updateTable(newMetadataLocation, oldMetadataLocation);
+      } else {
+        // table not exists create it
+        LOG.debug("Committing new table: {}", tableName());
+        createTable(newMetadataLocation);
+      }
+
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException(e, "Table already exists, maybe another process created it");
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Database Connection timeout");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Database Connection failed");
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException(e, "Database data truncation error");
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException(e, "Database warning");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to connect to database");

Review comment:
       I don't think that "failed to connect" is necessarily correct since the SQL exception is generic. How about "Unknown failure"?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537047495



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces {
+
+  public static final String JDBC_CATALOG_DEFAULT_NAME = "jdbc";
+
+  public static final String JDBC_CATALOG_JDBC_DRIVER = "jdbccatalog.jdbcdriver";
+  public static final String JDBC_CATALOG_DBURL = "jdbccatalog.dburl";
+  public static final String JDBC_CATALOG_USER = "jdbccatalog.user";
+  public static final String JDBC_CATALOG_PASSWORD = "jdbccatalog.password";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private QueryRunner queryRunner;
+  private FileIO fileIO;
+  private String catalogName = JDBC_CATALOG_DEFAULT_NAME;
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+
+
+  public JdbcCatalog() {
+  }
+
+  private void initializeCatalog() throws SQLException {
+    queryRunner.execute(JdbcNamespaceDao.NAMESPACES_TABLE_DDL);
+    queryRunner.execute(JdbcTableDao.JDBC_CATALOG_TABLE_DDL);
+    LOG.debug("Created Jdbc tables to store iceberg catalog!");
+  }
+
+  @Override
+  public void initialize(String name, Map<String, String> properties) {
+    Preconditions.checkArgument(properties.get(CatalogProperties.WAREHOUSE_LOCATION) != null &&
+                    !properties.get(CatalogProperties.WAREHOUSE_LOCATION).equals(""),
+            "no location provided for warehouse");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_JDBC_DRIVER, "").equals(""),
+            "no jdbc driver classname provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_DBURL, "").equals(""),
+            "no jdbc connection url provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_USER, "").equals(""),
+            "no jdbc database user provided!");
+    Preconditions.checkArgument(!properties.getOrDefault(JDBC_CATALOG_PASSWORD, "").equals(""),
+            "no jdbc database user password provided!");
+    this.catalogName = name;
+    this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION).replaceAll("/*$", "");
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.fileIO = fileIOImpl == null ? new HadoopFileIO(hadoopConf) : CatalogUtil.loadFileIO(fileIOImpl, properties,
+            hadoopConf);
+
+    LOG.debug("Connecting to Jdbc database {}.", properties.get(JDBC_CATALOG_DBURL));
+    // jdbcConnection = DriverManager.getConnection(catalogDburl, catalogUser, catalogDbpassword);

Review comment:
       changed to `DriverManager` and `Properties`. creating  Properties using prefix `connection.parameter.`
   




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r535746509



##########
File path: build.gradle
##########
@@ -226,6 +226,10 @@ project(':iceberg-core') {
     compile "com.fasterxml.jackson.core:jackson-databind"
     compile "com.fasterxml.jackson.core:jackson-core"
     compile "com.github.ben-manes.caffeine:caffeine"
+    compile 'commons-dbutils:commons-dbutils:1.7'
+    compile "org.hsqldb:hsqldb:2.5.1"
+    compile "mysql:mysql-connector-java:8.0.21"
+    compile "org.postgresql:postgresql:42.2.18"

Review comment:
       thought it will be nice to bundle common jdbc drivers, probably its not good idea removed mysql and postgres. checking point 4




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r550470807



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.sql.SQLWarning;
+import java.util.Properties;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+
+public class JdbcClientPool extends ClientPool<Connection, SQLException> {
+
+  private final String dbUrl;
+  private final Properties dbProperties;
+
+  JdbcClientPool(String dbUrl, Properties props) {

Review comment:
       i see updated it, 
   
   @rdblue what do you think about passing additional jdbc parameters with following prefix? is it okay or there shuld be no prefix?
   
   `public static final String JDBC_PARAM_PREFIX = "connection.parameter.";`
   
   current setup will be like 
   ```
   spark.sql.catalog.catalog_name.uri=jdbc:postgresql://localhost:5432/mydatabase
   spark.sql.catalog.catalog_name.connection.parameter.user=myuser
   spark.sql.catalog.catalog_name.connection.parameter.password=mypass
   spark.sql.catalog.catalog_name.connection.parameter.ssl=true
   spark.sql.catalog.catalog_name.connection.parameter.param1=val1
   spark.sql.catalog.catalog_name.connection.parameter.param2=val2
   ```




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r544661381



##########
File path: build.gradle
##########
@@ -231,6 +231,7 @@ project(':iceberg-core') {
       exclude group: 'org.slf4j', module: 'slf4j-log4j12'
     }
 
+    testRuntime "com.h2database:h2"

Review comment:
       I think `testCompile` is good enough, if we want to upgrade gradle we should upgrade all syntax at the same time.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+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.hadoop.Util;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+
+  public static final String JDBC_PARAM_PREFIX = "connection.parameter.";
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String TABLE_METADATA_FILE_EXTENSION = ".metadata.json";
+  private static final PathFilter TABLE_FILTER = path -> path.getName().endsWith(TABLE_METADATA_FILE_EXTENSION);
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private FileIO fileIO;
+  private FileSystem fs;
+  private String name = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection dbConn;
+
+  public JdbcCatalog(String name, Configuration conf, String warehouseLocation, Map<String, String> properties) {

Review comment:
       you need to also define a no-arg constructor for `initialize` for dynamic loading purpose. And in that case, you still need to initialize warehouse location and fileIO. So it is probably better to move all constructor logic to `initialize`.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+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.hadoop.Util;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+
+  public static final String JDBC_PARAM_PREFIX = "connection.parameter.";
+  private static final Joiner SLASH = Joiner.on("/");
+  private static final String TABLE_METADATA_FILE_EXTENSION = ".metadata.json";
+  private static final PathFilter TABLE_FILTER = path -> path.getName().endsWith(TABLE_METADATA_FILE_EXTENSION);
+
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class);
+  private FileIO fileIO;
+  private FileSystem fs;

Review comment:
       I don't understand why is `fs` used here. It seems like you have a `JdbcNamespace` that stores all the information, but also creates a file path for a namespace. Why are you doing this? I don't think the JdbcCatalog should have any dependency on Hadoop file system.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549527781



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.SQLNonTransientConnectionException;
+import java.sql.SQLTimeoutException;
+import java.sql.SQLTransientConnectionException;
+import java.sql.SQLWarning;
+import java.util.Properties;
+import org.apache.iceberg.ClientPool;
+import org.apache.iceberg.exceptions.UncheckedSQLException;
+
+public class JdbcClientPool extends ClientPool<Connection, SQLException> {
+
+  private final String dbUrl;
+  private final Properties dbProperties;
+
+  JdbcClientPool(String dbUrl, Properties props) {

Review comment:
       I don't think that Iceberg uses `Properties` anywhere, so this should probably accept `Map<String, String>` and pass that to JDBC as properties if necessary.
   
   Also, config properties don't need config context like `iceberg.jdbc` because that context is already dependent on how the catalog is configured. For example, Spark catalogs will use `spark.sql.catalog.catalog_name.uri` for the connection URI already. So the property keys here should just be the standard ones defined in [`CatalogProperties`](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/CatalogProperties.java).




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r550481916



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestClientPool.java
##########
@@ -19,6 +19,7 @@
 
 package org.apache.iceberg.hive;
 
+import org.apache.iceberg.ClientPool;

Review comment:
       my bad, thank you for pointing it.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549535476



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(namespace));
+      ResultSet rs = sql.executeQuery();
+
+      while (rs.next()) {
+        final TableIdentifier table = JdbcUtil.stringToTableIdentifier(
+                rs.getString("table_namespace"), rs.getString("table_name"));
+        results.add(table);
+      }
+      rs.close();
+      return results;
+
+    } catch (SQLException | InterruptedException e) {
+      LOG.error("Failed to list tables!", e);
+      return null;
+    }
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_UPDATE_TABLE_NAME));
+      // 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());
+      int updatedRecords = 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 UncheckedSQLException("Failed to rename table! Rename operation Failed");
+      }
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException("Table with name '%s' already exists in the catalog!", to);
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException("Database data truncation error!", e);
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException("Database warning!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to rename table!", e);
+    } catch (InterruptedException e) {
+      throw new UncheckedSQLException("Database Connection interrupted!", e);
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogName;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return this.hadoopConf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.hadoopConf = 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 (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace does not exist %s", namespace);
+    }
+    try {
+      List<Namespace> namespaces = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_NAMESPACES));
+      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()) {
+        rs.getString("table_namespace");
+        namespaces.add(JdbcUtil.stringToNamespace(rs.getString("table_namespace")));
+      }
+      rs.close();
+      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 (Exception e) {
+      LOG.error("Failed to list namespace!", e);
+      return null;
+    }
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    Path nsPath = new Path(warehouseLocation, JdbcUtil.JOINER_DOT.join(namespace.levels()));
+    if (!this.namespaceExists(namespace) || namespace.isEmpty()) {
+      throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace);
+    }
+
+    return ImmutableMap.of("location", nsPath.toString());
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Cannot drop namespace %s because it is not found!", namespace);
+    }
+
+    List<TableIdentifier> tableIdentifiers = listTables(namespace);
+    if (tableIdentifiers != null && !tableIdentifiers.isEmpty()) {
+      throw new NamespaceNotEmptyException("Cannot drop namespace %s because it is not empty. " +
+              "The following tables still exist under the namespace: %s", namespace, tableIdentifiers);
+    }
+    // namespaces are created/deleted by tables by default return true
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws
+          NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+            "Cannot set properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+            "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void close() {
+    this.dbConnPool.close();
+  }
+
+  public boolean namespaceExists(Namespace namespace) {

Review comment:
       Missing `@Override`.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549112734



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {

Review comment:
       > I'm not sure that I think this class is valuable. ...
   
   i see the pint totally makes sense. i see two options to address it 
   - merge it to `Catalog` and `TableOperations` class and only pass connection pool to `TableOperations`, similar to hive.
   - split it to `CatalogSQL`, `TableOperationsSQL` classes. `TableOperationsSQL` is used by `TableOperations`, and it additionally gets table identifier with constructor
   
   maybe its best to merge it to Catalog and address it with the future changes? to keep it simple for now.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537727184



##########
File path: LICENSE
##########
@@ -289,3 +289,15 @@ Copyright: 2011-2018 The Apache Software Foundation
 Home page: http://hive.apache.org/
 License: http://www.apache.org/licenses/LICENSE-2.0
 
+--------------------------------------------------------------------------------
+
+This software contains unmodified binary redistributions for
+H2 database engine (https://h2database.com/),
+which is dual licensed and available under the MPL 2.0
+(Mozilla Public License) or under the EPL 1.0 (Eclipse Public License).
+An original copy of the license agreement can be found at:
+https://h2database.com/html/license.html
+
+Copyright: Copyright 2004-2020 H2 Group. Multiple-Licensed under the MPL 2.0, and the EPL 1.0
+Home page: https://h2database.org/
+License: https://www.h2database.com/html/license.html

Review comment:
       This doesn't need to be added unless Iceberg actually contains something that is copyrighted by the H2 database engine. The LICENSE is updated if Iceberg adds code based on another project or if Iceberg bundles another project into an Iceberg artifact (like the runtime Jars). Since this is just used as a test dependency, there is no need.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547565476



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return !this.get(tableIdentifier).isEmpty();
+    } catch (SQLException | JsonProcessingException throwables) {
+      return false;
+    }
+  }
+
+  public Map<String, String> get(TableIdentifier tableIdentifier) throws SQLException, JsonProcessingException {
+    Map<String, String> table = Maps.newHashMap();
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));

Review comment:
       Rather than using JSON to encode the namespace, I suggest converting it to a String using `Joiner.on(".")`.
   
   There's a trade-off to simplifying the problem by doing it that way: both ```a.`b.c` ``` and `a.b.c` end up as `a.b.c`. That has two effects:
   1. Namespaces that "collide" like that are considered the same namespace
   2. When listing namespace `a` the result is `[b]` and not `[b.c, b]`
   
   I think that those are fine. For the collisions, I think it is rare for users to _want_ namespaces that can collide but are not the same. If someone creates the table ```a.`b.c`.table_name``` then I really doubt that everyone wants to escape `b.c` every time they type the identifier. Everyone is going to type `a.b.c.table_name` anyway, which would return incorrect results if both tables exist. So not allowing namespaces that conflict that that is actually a good thing.
   
   This is exactly what Iceberg does to look up columns in a table, too. All column names are converted to a lookup key by joining the parts with `.`. Columns that have different structures but identical names aren't allowed, and I think it is better that way.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536893335



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;

Review comment:
       How about `org.apache.iceberg.jdbc`? I'm okay with `catalogs` as well.
   
   As for the module, I'm okay putting this in core for now. Let's see how big it ends up being and we can move it when we know more. I think that if it has a small enough dependency footprint, we can keep it here like we talked about in the recent community sync. If it is small, then it is much easier for people to have it included in our runtime builds by default.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536895844



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;

Review comment:
       It doesn't look like the dbutils classes are very useful here compared to simple prepared statements. Unless I'm missing something, I'd rather not rely on this dependency if we can add slightly more code here to use the JDBC API directly.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r547568504



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+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.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable {
+
+  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 name = "jdbc";
+  private String warehouseLocation;
+  private Configuration hadoopConf;
+  private Connection dbConn;
+
+  public JdbcCatalog() {
+  }
+
+  @SuppressWarnings("checkstyle:HiddenField")
+  @Override
+  public void initialize(String name, Map<String, String> properties) throws UncheckedIOException {
+    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.name = 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) throws UncheckedIOException {
+    try {
+
+      LOG.debug("Connecting to Jdbc database {}.", properties.get(CatalogProperties.HIVE_URI));
+      Properties connectionProps = new Properties();
+      for (Map.Entry<String, String> prop : properties.entrySet()) {
+        if (prop.getKey().startsWith(JDBC_PARAM_PREFIX)) {
+          connectionProps.put(prop.getKey().substring(JDBC_PARAM_PREFIX.length()), prop.getValue());
+        }
+      }
+
+      dbConn = DriverManager.getConnection(properties.get(CatalogProperties.HIVE_URI), connectionProps);
+      initializeCatalogTables();
+
+    } catch (SQLException throwables) {

Review comment:
       Style: just one exception is caught, so the variable name should be singular, like `throwable`. Also, we tend to refer to the exception as simply `e` in most catch blocks.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r548243345



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {
+  public static final String SQL_TABLE_NAME = "iceberg_tables";
+  public static final String SQL_TABLE_DDL =
+          "CREATE TABLE " + JdbcTable.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 = "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_METADATA_LOCATION = "UPDATE " + SQL_TABLE_NAME +
+          " SET metadata_location = ? , previous_metadata_location = ? " +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? AND metadata_location = ?";
+  public static final String SQL_INSERT = "INSERT INTO " + SQL_TABLE_NAME +
+          " (catalog_name, table_namespace, table_name, metadata_location, previous_metadata_location) " +
+          " VALUES (?,?,?,?,?)";
+  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 = "DELETE FROM " + SQL_TABLE_NAME +
+          " WHERE catalog_name = ? AND table_namespace = ? AND table_name = ? ";
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTable.class);
+  private final String catalogName;
+  private final Connection dbConn;
+
+  public JdbcTable(Connection dbConn, String catalogName) {
+    this.dbConn = dbConn;
+    this.catalogName = catalogName;
+  }
+
+  public void setAutoCommitOff() throws SQLException {
+    this.dbConn.setAutoCommit(false);
+  }
+
+  public void setAutoCommitOn() throws SQLException {
+    this.dbConn.setAutoCommit(true);
+  }
+
+  public void commit() throws SQLException {
+    this.dbConn.commit();
+  }
+
+  public void rollback() throws SQLException {
+    this.dbConn.rollback();
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return !this.get(tableIdentifier).isEmpty();
+    } catch (SQLException | JsonProcessingException throwables) {
+      return false;
+    }
+  }
+
+  public Map<String, String> get(TableIdentifier tableIdentifier) throws SQLException, JsonProcessingException {
+    Map<String, String> table = Maps.newHashMap();
+    PreparedStatement sql = dbConn.prepareStatement(SQL_SELECT);
+    sql.setString(1, catalogName);
+    sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace()));

Review comment:
       changed to Joiner




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


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

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r636610991



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,394 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    if (name != null) {
+      this.catalogName = name;
+    }
+
+    String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL);
+    this.io = fileIOImpl == null ?
+        new HadoopFileIO(conf) :

Review comment:
       nit: no need for newline here

##########
File path: core/src/main/java/org/apache/iceberg/CatalogUtil.java
##########
@@ -48,8 +48,10 @@
   public static final String ICEBERG_CATALOG_TYPE = "type";
   public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop";
   public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive";
+  public static final String ICEBERG_CATALOG_TYPE_JDBC = "jdbc";

Review comment:
       I don't think we need all the logic in this file, it's much cleaner for everything other than hive and hadoop to be loaded through impl. It makes the user experience much more consistent, and we don't need to keep updating the list of types in documentation.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,394 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException("Database Connection timeout", e);

Review comment:
       nit: error message should follow format "Cannot xxx: reason"

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/UncheckedSQLException.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.SQLException;
+
+public class UncheckedSQLException extends RuntimeException {
+
+  public UncheckedSQLException(String message, SQLException cause) {

Review comment:
       should add format constructors following the pattern of other Iceberg exceptions, see `ValidationException` for more details.

##########
File path: spark3/src/test/java/org/apache/iceberg/actions/TestRemoveOrphanFilesAction3.java
##########
@@ -165,4 +165,64 @@ public void testSparkSessionCatalogHiveTable() throws Exception {
     spark.conf().unset("spark.sql.catalog.spark_catalog.type");
   }
 
+  @Test
+  public void testSparkCatalogNamedJdbcTable() throws Exception {
+    spark.conf().set("spark.sql.catalog.jdbc", "org.apache.iceberg.spark.SparkCatalog");
+    spark.conf().set("spark.sql.catalog.jdbc.type", "jdbc");
+    spark.conf().set("spark.sql.catalog.jdbc.warehouse", tableLocation);
+    spark.conf().set("spark.sql.catalog.jdbc.uri", "jdbc:sqlite:file::memory:?icsparkjdbctestcat;");
+    spark.conf().set("spark.sql.catalog.jdbc.connection.parameter.user", "testuser");
+    spark.conf().set("spark.sql.catalog.jdbc.connection.parameter.password", "testpassword");
+    SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("jdbc");
+
+    String[] database = {"default"};
+    Identifier id = Identifier.of(database, "table");
+    Map<String, String> options = Maps.newHashMap();
+    Transform[] transforms = {};
+    cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options);
+    SparkTable table = cat.loadTable(id);
+
+    spark.sql("INSERT INTO jdbc.default.table VALUES (1,1,1)");
+
+    String location = table.table().location().replaceFirst("file:", "");
+    new File(location + "/data/trashfile").createNewFile();
+
+    List<String> results = Actions.forTable(table.table()).removeOrphanFiles()
+        .olderThan(System.currentTimeMillis() + 1000).execute();
+    Assert.assertTrue("trash file should be removed",
+        results.contains("file:" + location + "/data/trashfile"));
+  }
+
+  @Test
+  public void testSparkSessionCatalogJdbcTable() throws Exception {
+    spark.conf().set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog");
+    spark.conf().set("spark.sql.catalog.spark_catalog.type", "jdbc");
+    spark.conf().set("spark.sql.catalog.spark_catalog.warehouse", tableLocation);
+    spark.conf().set("spark.sql.catalog.spark_catalog.uri", "jdbc:sqlite:file::memory:?icsparktestcat;");
+    spark.conf().set("spark.sql.catalog.spark_catalog.connection.parameter.user", "testuser");
+    spark.conf().set("spark.sql.catalog.spark_catalog.connection.parameter.password", "testpassword");
+    SparkSessionCatalog cat = (SparkSessionCatalog) spark.sessionState().catalogManager().v2SessionCatalog();
+
+    String[] database = {"default"};
+    Identifier id = Identifier.of(database, "sessioncattest");
+    Map<String, String> options = Maps.newHashMap();
+    Transform[] transforms = {};
+    cat.dropTable(id);
+    cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options);
+    SparkTable table = (SparkTable) cat.loadTable(id);
+
+    spark.sql("INSERT INTO default.sessioncattest VALUES (1,1,1)");
+
+    String location = table.table().location().replaceFirst("file:", "");
+    new File(location + "/data/trashfile").createNewFile();
+
+    List<String> results = Actions.forTable(table.table()).removeOrphanFiles()
+        .olderThan(System.currentTimeMillis() + 1000).execute();
+    Assert.assertTrue("trash file should be removed",
+        results.contains("file:" + location + "/data/trashfile"));
+    // reset spark_catalog to default

Review comment:
       will the test fail if we don't set it back? If not we can remove it, if so we should have a `@After` to clean this up.

##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,394 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } 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 initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating tables(if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {

Review comment:
       no need for if condition.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637616682



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+

Review comment:
       Nit: we don't typically start a method with an empty line.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r655724859



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class JdbcTableOperations extends BaseMetastoreTableOperations {
+
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class);
+  private final String catalogName;
+  private final TableIdentifier tableIdentifier;
+  private final FileIO fileIO;
+  private final JdbcClientPool connections;
+
+  protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName,
+                                TableIdentifier tableIdentifier) {
+    this.catalogName = catalogName;
+    this.tableIdentifier = tableIdentifier;
+    this.fileIO = fileIO;
+    this.connections = dbConnPool;
+  }
+
+  @Override
+  public void doRefresh() {
+    Map<String, String> table;
+
+    try {
+      table = getTable();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new UncheckedInterruptedException(e, "Interrupted during refresh");
+    } catch (SQLException e) {
+      // SQL exception happened when getting table from catalog
+      throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName);
+    }
+
+    // Table not exists AND currentMetadataLocation is not NULL!
+    if (table.isEmpty() && currentMetadataLocation() != null) {
+      throw new NoSuchTableException("Failed to get table %s from catalog %s" +
+          " maybe another process deleted it", tableIdentifier, catalogName);
+    }
+
+    // Table not exists in the catalog! metadataLocation is null here!
+    if (table.isEmpty()) {
+      refreshFromMetadataLocation(null);

Review comment:
       Nice job catching and fixing that.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r562118168



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




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r535731569



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.hadoop;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.dbutils.DbUtils;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+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.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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, Closeable {

Review comment:
       working on it now




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549112734



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {

Review comment:
       > I'm not sure that I think this class is valuable. ...
   
   i see the pint totally makes sense. i see two options to address it 
   - merge it to `Catalog` and `TableOperations` class and only pass connection pool to `TableOperations`, similar to hive.
   - split it to `CatalogSQL`, `TableOperationsSQL` classes. `TableOperationsSQL` is used by `TableOperations`, and it additionally gets table identifier with constructor
   
   maybe its best to merge it to Catalog and address it with the future changes?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537605790



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +
+                  "(catalogName, namespace)" +
+                  ");";
+  private static final Logger LOG = LoggerFactory.getLogger(JdbcTableDao.class);
+  private final QueryRunner queryRunner;
+  private final String catalogName;
+
+
+  public JdbcTableDao(QueryRunner queryRunner, String catalogName) {
+    this.queryRunner = queryRunner;
+    this.catalogName = catalogName;
+  }
+
+  public boolean isExists(TableIdentifier tableIdentifier) {
+    try {
+      return this.get(tableIdentifier) != null;
+    } catch (SQLException throwables) {

Review comment:
       i would expect potential exceptions like 'when the database is down', 'not reachable' or 'table is not there'. otherwise the result shuld return = null 
   maybe its better to re-throw the exception? otherwise catalog class will assume table is not exists and will continue.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536670800



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcCatalog.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.hadoop;

Review comment:
       I based the implementation using Hadoop catalog as an example(and glue catalog) so started it here, Maybe new module named something like "catalog" more suitable?




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r553455161



##########
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:
       with [earlier discussion](https://github.com/apache/iceberg/pull/1870#discussion_r547560663
   ) we agreed to keep initial implementation simple and based on single table. 
   [another point about namespace metadata](https://github.com/apache/iceberg/pull/1870#discussion_r536904152)
   
   with current implementatin if we want to implement `createNamespace` we need to create a table with it because its part of the primary key `PRIMARY KEY (catalog_name, table_namespace, table_name)`




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637402335



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,394 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } 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 initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating tables(if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {

Review comment:
       good pint, removed if condition




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r536905416



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class JdbcTableDao {
+  public static final String TABLES_TABLE_NAME = "iceberg_tables";
+  public static final String JDBC_CATALOG_TABLE_DDL =
+          "CREATE TABLE IF NOT EXISTS " + JdbcTableDao.TABLES_TABLE_NAME +
+                  "(catalogName VARCHAR(1255) NOT NULL," +
+                  "tableNamespace VARCHAR(1255) NOT NULL," +
+                  "tableName VARCHAR(1255) NOT NULL," +
+                  "metadataLocation VARCHAR(32768)," +
+                  "previousMetadataLocation VARCHAR(32768)," +
+                  "PRIMARY KEY (catalogName,tableNamespace,tableName)," +
+                  "FOREIGN KEY (catalogName, tableNamespace) REFERENCES " + JdbcNamespaceDao.NAMESPACES_TABLE_NAME +

Review comment:
       Is this `PRIMARY KEY` and `FOREIGN KEY` syntax portable? I'm wondering if we should omit them to have more generic SQL.
   
   Of course, that would require some way to ensure that table creation is atomic because the primary key constraint wouldn't be enforced for the table. That would probably require a transaction to check whether a table exists and insert it to be portable. Some databases support `ON CONFLICT DO NOTHING` or `INSERT IGNORE` but it doesn't look like the syntax is portable.
   
   It may be more portable to use `PRIMARY KEY`, but we will need to validate concurrent behavior.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r537046104



##########
File path: core/src/main/java/org/apache/iceberg/hadoop/JdbcTableDao.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.hadoop;
+
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.commons.dbutils.QueryRunner;
+import org.apache.commons.dbutils.handlers.BeanHandler;
+import org.apache.commons.dbutils.handlers.BeanListHandler;

Review comment:
       removed `apache.commons` db packages. refactored code its using only JDBC API now, code-size even got smaller




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


[GitHub] [iceberg] rdblue commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-865367950


   Thanks, @ismailsimsek for working on this! I just merged it.
   
   And thanks to @jackye1995, @kbendick, and @massdosage for reviewing!


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637619422



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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(JdbcUtil.LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.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
+          .filter(n -> n.levels().length >= subNamespaceLevelLength)
+          // 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);
+    }
+
+    return ImmutableMap.of("location", defaultNamespaceLocation(namespace));
+  }
+
+  private String defaultNamespaceLocation(Namespace namespace) {
+    if (namespace.isEmpty()) {
+      return warehouseLocation;
+    } else {
+      return SLASH.join(warehouseLocation, SLASH.join(namespace.levels()));
+    }
+  }
+
+  @Override
+  public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException {
+    if (!namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Cannot drop namespace %s because it is not found!", namespace);
+    }
+
+    List<TableIdentifier> tableIdentifiers = listTables(namespace);
+    if (tableIdentifiers != null && !tableIdentifiers.isEmpty()) {
+      throw new NamespaceNotEmptyException("Cannot drop namespace %s because it is not empty. " +
+          "Namespace contains %s tables", namespace, tableIdentifiers.size());
+    }
+
+    // namespaces are created/deleted by tables by default return true
+    // when there is no tables with namespace then its considered dropped
+    return true;
+  }
+
+  @Override
+  public boolean setProperties(Namespace namespace, Map<String, String> properties) throws
+      NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot set properties " + namespace + " : setProperties is not supported");
+  }
+
+  @Override
+  public boolean removeProperties(Namespace namespace, Set<String> properties) throws NoSuchNamespaceException {
+    throw new UnsupportedOperationException(
+        "Cannot remove properties " + namespace + " : removeProperties is not supported");
+  }
+
+  @Override
+  public void close() {
+    connections.close();
+  }
+
+  @Override
+  public boolean namespaceExists(Namespace namespace) {
+    try {
+      return connections.run(conn -> {
+        boolean exists = false;
+
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.GET_NAMESPACE_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          if (rs.next()) {
+            exists = true;
+          }
+
+          rs.close();
+        }
+
+        return exists;
+      });
+
+    } catch (SQLException e) {
+      LOG.warn("SQLException! ", e);
+      return false;

Review comment:
       I think this needs to throw `UncheckedSQLException`. It can't assume that any exception indicates that the table doesn't exist.




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


[GitHub] [iceberg] ismailsimsek commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-784154992


   sorry i got busy with some other stuff, it should be close to finish, @XuQianJin-Stars please feel free to add your review @rdblue  @jackye1995  its ready for another review


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r637618484



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java
##########
@@ -0,0 +1,388 @@
+/*
+ * 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) {
+    String uri = properties.get(CatalogProperties.URI);
+    Preconditions.checkNotNull(uri, "JDBC connection URI is required");
+
+    String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION);
+    Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required");
+    this.warehouseLocation = warehouse.replaceAll("/$", "");
+
+    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 {
+      LOG.debug("Connecting to Jdbc database {}", properties.get(CatalogProperties.URI));
+      connections = new JdbcClientPool(uri, properties);
+      initializeCatalogTables();
+    } catch (SQLTimeoutException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Query timed out");
+    } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog: Connection failed");
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Cannot initialize jdbc catalog");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to initialize", e);
+    }
+  }
+
+  private void initializeCatalogTables() throws InterruptedException, SQLException {
+    LOG.trace("Creating database tables (if missing) to store iceberg catalog");
+    connections.run(conn -> {
+      DatabaseMetaData dbMeta = conn.getMetaData();
+      ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null);
+
+      if (tableExists.next()) {
+        return true;
+      }
+
+      LOG.debug("Creating table {} to store iceberg catalog!", JdbcUtil.CATALOG_TABLE_NAME);
+      return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute();
+    });
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(connections, io, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+
+    int deletedRecords;
+    try {
+      deletedRecords = connections.run(conn -> {
+        try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.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(e, "Failed to drop %s", identifier);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Interrupted in call to dropTable", e);
+    }
+
+    if (deletedRecords > 0) {
+      LOG.info("Successfully dropped table {}.", identifier);
+    } else {
+      LOG.info("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(JdbcUtil.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(JdbcUtil.TABLE_NAMESPACE), rs.getString(
+                JdbcUtil.TABLE_NAME)));
+          }
+
+          return results;
+        }
+      });
+
+    } catch (SQLException e) {
+      throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace);
+    } 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(JdbcUtil.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(e, "Failed to rename table");
+    } 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(JdbcUtil.LIST_NAMESPACES_SQL)) {
+          sql.setString(1, catalogName);
+          sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%");
+          ResultSet rs = sql.executeQuery();
+          while (rs.next()) {
+            result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.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
+          .filter(n -> n.levels().length >= subNamespaceLevelLength)
+          // 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);

Review comment:
       I prefer not to add logs like this that take up so much space. Debug is okay, but in general I would avoid logs that simply state what happened (and in verbose detail) during normal execution unless it is a significant state change for the DB (like renamed, dropped, or created).




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


[GitHub] [iceberg] ismailsimsek commented on pull request #1870: Iceberg Jdbc Catalog Implementation

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#issuecomment-760354791


   Thank you for the review @jackye1995 its ready for another look.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549535388



##########
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
+    boolean exists = false;
+    DatabaseMetaData dbMeta = dbConnPool.run(Connection::getMetaData);
+    ResultSet tables = dbMeta.getTables(null, null, SQL_TABLE_NAME, null);
+    if (tables.next()) {
+      exists = true;
+    }
+    tables.close();
+    ResultSet tablesUpper = dbMeta.getTables(null, null, SQL_TABLE_NAME.toUpperCase(), null);
+    if (tablesUpper.next()) {
+      exists = true;
+    }
+    tablesUpper.close();
+    ResultSet tablesLower = dbMeta.getTables(null, null, SQL_TABLE_NAME.toLowerCase(), null);
+    if (tablesLower.next()) {
+      exists = true;
+    }
+    tablesLower.close();
+
+    // create table if not exits
+    if (!exists) {
+      dbConnPool.run(conn -> conn.prepareStatement(SQL_TABLE_DDL).execute());
+      LOG.debug("Created table {} to store iceberg tables!", SQL_TABLE_NAME);
+    }
+  }
+
+  @Override
+  protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
+    return new JdbcTableOperations(dbConnPool, fileIO, catalogName, tableIdentifier);
+  }
+
+  @Override
+  protected String defaultWarehouseLocation(TableIdentifier table) {
+    if (table.hasNamespace()) {
+      return SLASH.join(warehouseLocation, table.namespace().levels(), table.name());
+    }
+    return SLASH.join(warehouseLocation, table.name());
+  }
+
+  @Override
+  public boolean dropTable(TableIdentifier identifier, boolean purge) {
+    TableOperations ops = newTableOps(identifier);
+    TableMetadata lastMetadata = ops.current();
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_DELETE_TABLE));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace()));
+      sql.setString(3, identifier.name());
+      int deletedRecords = sql.executeUpdate();
+
+      if (deletedRecords > 0) {
+        LOG.debug("Successfully dropped table {}.", identifier);
+      } else {
+        throw new NoSuchTableException("Cannot drop table %s! table not found in the catalog.", identifier);
+      }
+
+      if (purge && lastMetadata != null) {
+        CatalogUtil.dropTableData(ops.io(), lastMetadata);
+        LOG.info("Table {} data purged!", identifier);
+      }
+      return true;
+    } catch (SQLException | InterruptedException e) {
+      throw new UncheckedSQLException("Failed to drop table!", e);
+    }
+  }
+
+  @Override
+  public List<TableIdentifier> listTables(Namespace namespace) {
+    if (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace %s does not exist!", namespace);
+    }
+    try {
+      List<TableIdentifier> results = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_ALL));
+      sql.setString(1, catalogName);
+      sql.setString(2, JdbcUtil.namespaceToString(namespace));
+      ResultSet rs = sql.executeQuery();
+
+      while (rs.next()) {
+        final TableIdentifier table = JdbcUtil.stringToTableIdentifier(
+                rs.getString("table_namespace"), rs.getString("table_name"));
+        results.add(table);
+      }
+      rs.close();
+      return results;
+
+    } catch (SQLException | InterruptedException e) {
+      LOG.error("Failed to list tables!", e);
+      return null;
+    }
+  }
+
+  @Override
+  public void renameTable(TableIdentifier from, TableIdentifier to) {
+    try {
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_UPDATE_TABLE_NAME));
+      // 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());
+      int updatedRecords = 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 UncheckedSQLException("Failed to rename table! Rename operation Failed");
+      }
+    } catch (SQLIntegrityConstraintViolationException e) {
+      throw new AlreadyExistsException("Table with name '%s' already exists in the catalog!", to);
+    } catch (DataTruncation e) {
+      throw new UncheckedSQLException("Database data truncation error!", e);
+    } catch (SQLWarning e) {
+      throw new UncheckedSQLException("Database warning!", e);
+    } catch (SQLException e) {
+      throw new UncheckedSQLException("Failed to rename table!", e);
+    } catch (InterruptedException e) {
+      throw new UncheckedSQLException("Database Connection interrupted!", e);
+    }
+  }
+
+  @Override
+  public String name() {
+    return catalogName;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return this.hadoopConf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.hadoopConf = 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 (!this.namespaceExists(namespace)) {
+      throw new NoSuchNamespaceException("Namespace does not exist %s", namespace);
+    }
+    try {
+      List<Namespace> namespaces = Lists.newArrayList();
+      PreparedStatement sql = dbConnPool.run(c -> c.prepareStatement(SQL_SELECT_NAMESPACES));
+      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()) {
+        rs.getString("table_namespace");
+        namespaces.add(JdbcUtil.stringToNamespace(rs.getString("table_namespace")));
+      }
+      rs.close();
+      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 (Exception e) {
+      LOG.error("Failed to list namespace!", e);
+      return null;
+    }
+  }
+
+  @Override
+  public Map<String, String> loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException {
+    Path nsPath = new Path(warehouseLocation, JdbcUtil.JOINER_DOT.join(namespace.levels()));

Review comment:
       Shouldn't this use `SLASH` for the path instead?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549529800



##########
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);

Review comment:
       I think these exceptions should be handled in `initializeCatalogTables`, not in this method.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549528369



##########
File path: hive-metastore/src/test/java/org/apache/iceberg/hive/TestClientPool.java
##########
@@ -19,6 +19,7 @@
 
 package org.apache.iceberg.hive;
 
+import org.apache.iceberg.ClientPool;

Review comment:
       This test should also be moved if the original class was moved.




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


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

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r549148181



##########
File path: core/src/main/java/org/apache/iceberg/jdbc/JdbcTable.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 com.fasterxml.jackson.core.JsonProcessingException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JdbcTable {

Review comment:
       went with first option and moved code to to `Catalog` and `TableOperations` classes.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_r562300208



##########
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 `!`




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