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



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

Reply via email to