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



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

Reply via email to