You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/05/21 03:44:58 UTC

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

jackye1995 commented on a change in pull request #1870:
URL: https://github.com/apache/iceberg/pull/1870#discussion_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