JingsongLi commented on code in PR #831: URL: https://github.com/apache/incubator-paimon/pull/831#discussion_r1159304665
########## paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactoryOptions.java: ########## @@ -0,0 +1,44 @@ +/* + * 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.paimon.hive; + +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.options.ConfigOptions; + +/** Options for hive catalog. */ +public final class HiveCatalogFactoryOptions { Review Comment: `HiveCatalogOptions`? ########## docs/layouts/shortcodes/generated/hive_catalog_factory_configuration.html: ########## @@ -0,0 +1,24 @@ +<table class="configuration table table-bordered"> Review Comment: Modify `maintenance/configurations` too. ########## docs/content/how-to/creating-catalogs.md: ########## @@ -84,12 +84,15 @@ Paimon Hive catalog in Flink relies on Flink Hive connector bundled jar. You sho The following Flink SQL registers and uses a Paimon Hive catalog named `my_hive`. Metadata and table files are stored under `hdfs://path/to/warehouse`. In addition, metadata is also stored in Hive metastore. +If your Hive requires security authentication such as Kerberos,LDAP,Ranger and so on. You can specify the hive-conf-dir parameter to the hive-site.xml file path + ```sql CREATE CATALOG my_hive WITH ( 'type' = 'paimon', 'metastore' = 'hive', 'uri' = 'thrift://<hive-metastore-host-name>:<port>', - 'warehouse' = 'hdfs://path/to/warehouse' + 'warehouse' = 'hdfs://path/to/warehouse', + 'hive-conf-dir' = 'hdfs://path/to/hive-conf', Review Comment: No need to modify here. Only has Kerberos.. requirement, user configure this. ########## docs/content/how-to/creating-catalogs.md: ########## @@ -84,12 +84,15 @@ Paimon Hive catalog in Flink relies on Flink Hive connector bundled jar. You sho The following Flink SQL registers and uses a Paimon Hive catalog named `my_hive`. Metadata and table files are stored under `hdfs://path/to/warehouse`. In addition, metadata is also stored in Hive metastore. +If your Hive requires security authentication such as Kerberos,LDAP,Ranger and so on. You can specify the hive-conf-dir parameter to the hive-site.xml file path Review Comment: `If your Hive requires security authentication such as Kerberos, LDAP, Ranger and so on. You can specify the hive-conf-dir parameter to the hive-site.xml file path.` ########## paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java: ########## @@ -521,9 +528,88 @@ static IMetaStoreClient createClient(HiveConf hiveConf, String clientClassName) } catch (Exception e) { throw new RuntimeException(e); } - return StringUtils.isNullOrWhitespaceOnly( - hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)) + return isNullOrWhitespaceOnly(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)) ? client : HiveMetaStoreClient.newSynchronizedClient(client); } + + public static HiveConf createHiveConf( + @Nullable String hiveConfDir, @Nullable String hadoopConfDir) { + // create HiveConf from hadoop configuration with hadoop conf directory configured. + Configuration hadoopConf = null; + if (!isNullOrWhitespaceOnly(hadoopConfDir)) { + hadoopConf = getHadoopConfiguration(hadoopConfDir); + if (hadoopConf == null) { + String possiableUsedConfFiles = + "core-site.xml | hdfs-site.xml | yarn-site.xml | mapred-site.xml"; + throw new RuntimeException( + "Failed to load the hadoop conf from specified path:" + hadoopConfDir, + new FileNotFoundException( + "Please check the path none of the conf files (" + + possiableUsedConfFiles + + ") exist in the folder.")); + } + } + if (hadoopConf == null) { + hadoopConf = new Configuration(); + } + // ignore all the static conf file URLs that HiveConf may have set + HiveConf.setHiveSiteLocation(null); + HiveConf.setLoadMetastoreConfig(false); + HiveConf.setLoadHiveServer2Config(false); + HiveConf hiveConf = new HiveConf(hadoopConf, HiveConf.class); + + LOG.info("Setting hive conf dir as {}", hiveConfDir); + + if (hiveConfDir != null) { + org.apache.hadoop.fs.Path hiveSite = + new org.apache.hadoop.fs.Path(hiveConfDir, HIVE_SITE_FILE); + if (!hiveSite.toUri().isAbsolute()) { + hiveSite = new org.apache.hadoop.fs.Path(new File(hiveSite.toString()).toURI()); Review Comment: I mean why here must be a local 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
