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: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
