danny0405 commented on code in PR #6082: URL: https://github.com/apache/hudi/pull/6082#discussion_r920736196
########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java: ########## @@ -0,0 +1,897 @@ +/* + * 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.hudi.table.catalog; + +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; +import org.apache.hudi.exception.HoodieCatalogException; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.sync.common.util.ConfigUtils; +import org.apache.hudi.table.format.FilePathUtils; +import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.avro.Schema; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase; +import org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner; +import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveDatabase; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogPropertiesUtil; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogView; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.PrincipalType; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabase.ALTER_DATABASE_OP; +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner.DATABASE_OWNER_NAME; +import static org.apache.flink.sql.parser.hive.ddl.SqlAlterHiveDatabaseOwner.DATABASE_OWNER_TYPE; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; +import static org.apache.hudi.configuration.FlinkOptions.PATH; +import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DB; +import static org.apache.hudi.table.catalog.TableOptionProperties.COMMENT; +import static org.apache.hudi.table.catalog.TableOptionProperties.PK_COLUMNS; +import static org.apache.hudi.table.catalog.TableOptionProperties.PK_CONSTRAINT_NAME; +import static org.apache.hudi.table.catalog.TableOptionProperties.SPARK_SOURCE_PROVIDER; + +/** + * A catalog implementation for Hoodie based on MetaStore. + */ +public class HoodieHiveCatalog extends AbstractCatalog { + private static final Logger LOG = LoggerFactory.getLogger(HoodieHiveCatalog.class); + + private final HiveConf hiveConf; + private IMetaStoreClient client; + + public HoodieHiveCatalog(String catalogName, String defaultDatabase, String hiveConf) { + this(catalogName, defaultDatabase, HoodieCatalogUtil.createHiveConf(hiveConf), false); + } + + public HoodieHiveCatalog(String catalogName, String defaultDatabase, HiveConf hiveConf, boolean allowEmbedded) { + super(catalogName, defaultDatabase == null ? DEFAULT_DB : defaultDatabase); + this.hiveConf = hiveConf; + if (!allowEmbedded) { + checkArgument( + !HoodieCatalogUtil.isEmbeddedMetastore(this.hiveConf), + "Embedded metastore is not allowed. Make sure you have set a valid value for " + + HiveConf.ConfVars.METASTOREURIS); + } + LOG.info("Created Hoodie Catalog '{}' in hms mode", catalogName); + } + + @Override + public void open() throws CatalogException { + if (this.client == null) { + try { + this.client = Hive.get(hiveConf).getMSC(); + } catch (Exception e) { + throw new HoodieCatalogException("Failed to create hive metastore client", e); + } + LOG.info("Connected to Hive metastore"); + } + if (!databaseExists(getDefaultDatabase())) { + LOG.info("{} does not exist, will be created.", getDefaultDatabase()); + CatalogDatabase database = new CatalogDatabaseImpl(Collections.EMPTY_MAP, "default database"); + try { + createDatabase(getDefaultDatabase(), database, true); + } catch (DatabaseAlreadyExistException e) { + throw new HoodieCatalogException(getName(), e); + } + } + } + + @Override + public void close() throws CatalogException { + if (client != null) { + client.close(); + client = null; + LOG.info("Disconnect to hive metastore"); + } + } + + public HiveConf getHiveConf() { + return hiveConf; + } + + // ------ databases ------ + + @Override + public List<String> listDatabases() throws CatalogException { + try { + return client.getAllDatabases(); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to list all databases in %s", getName()), e); + } + } + + private Database getHiveDatabase(String databaseName) throws DatabaseNotExistException { + try { + return client.getDatabase(databaseName); + } catch (NoSuchObjectException e) { + throw new DatabaseNotExistException(getName(), databaseName); + } catch (TException e) { + throw new HoodieCatalogException( + String.format("Failed to get database %s from %s", databaseName, getName()), e); + } + } + + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + Database hiveDatabase = getHiveDatabase(databaseName); + + Map<String, String> properties = new HashMap<>(hiveDatabase.getParameters()); + + properties.put(SqlCreateHiveDatabase.DATABASE_LOCATION_URI, hiveDatabase.getLocationUri()); + + return new CatalogDatabaseImpl(properties, hiveDatabase.getDescription()); + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + try { + return client.getDatabase(databaseName) != null; + } catch (NoSuchObjectException e) { + return false; + } catch (TException e) { + throw new HoodieCatalogException( + String.format( + "Failed to determine whether database %s exists or not", databaseName), + e); + } + } + + @Override + public void createDatabase( + String databaseName, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + checkArgument( + !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or empty"); + checkNotNull(database, "database cannot be null"); Review Comment: `databaseName cannot` -> `Database name can not`, also fix the following error messages. -- 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]
