cuibo01 commented on code in PR #6082:
URL: https://github.com/apache/hudi/pull/6082#discussion_r922218747


##########
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");
+
+    Map<String, String> properties = database.getProperties();
+
+    String dbLocationUri = 
properties.remove(SqlCreateHiveDatabase.DATABASE_LOCATION_URI);
+
+    Database hiveDatabase =
+        new Database(databaseName, database.getComment(), dbLocationUri, 
properties);
+
+    try {
+      client.createDatabase(hiveDatabase);
+    } catch (AlreadyExistsException e) {
+      if (!ignoreIfExists) {
+        throw new DatabaseAlreadyExistException(getName(), 
hiveDatabase.getName());
+      }
+    } catch (TException e) {
+      throw new HoodieCatalogException(
+          String.format("Failed to create database %s", 
hiveDatabase.getName()), e);
+    }
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean ignoreIfNotExists, boolean 
cascade)
+      throws DatabaseNotExistException, DatabaseNotEmptyException, 
CatalogException {
+    try {
+      client.dropDatabase(name, true, ignoreIfNotExists, cascade);
+    } catch (NoSuchObjectException e) {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), name);
+      }
+    } catch (InvalidOperationException e) {
+      throw new DatabaseNotEmptyException(getName(), name);
+    } catch (TException e) {
+      throw new HoodieCatalogException(String.format("Failed to drop database 
%s", name), e);
+    }
+  }
+
+  @Override
+  public void alterDatabase(
+      String databaseName, CatalogDatabase newDatabase, boolean 
ignoreIfNotExists)
+      throws DatabaseNotExistException, CatalogException {
+    checkArgument(
+        !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or 
empty");
+    checkNotNull(newDatabase, "newDatabase cannot be null");
+
+    // client.alterDatabase doesn't throw any exception if there is no 
existing database
+    Database hiveDB;
+    try {
+      hiveDB = getHiveDatabase(databaseName);
+    } catch (DatabaseNotExistException e) {
+      if (!ignoreIfNotExists) {
+        throw new DatabaseNotExistException(getName(), databaseName);
+      }
+
+      return;
+    }
+
+    try {
+      client.alterDatabase(databaseName, alterDatabase(hiveDB, newDatabase));
+    } catch (TException e) {
+      throw new HoodieCatalogException(
+          String.format("Failed to alter database %s", databaseName), e);
+    }
+  }
+
+  private static Database alterDatabase(Database hiveDB, CatalogDatabase 
newDatabase) {
+    Map<String, String> newParams = newDatabase.getProperties();
+    String opStr = newParams.remove(ALTER_DATABASE_OP);
+    if (opStr == null) {
+      // by default is to alter db properties
+      opStr = SqlAlterHiveDatabase.AlterHiveDatabaseOp.CHANGE_PROPS.name();
+    }
+    String newLocation = 
newParams.remove(SqlCreateHiveDatabase.DATABASE_LOCATION_URI);
+    SqlAlterHiveDatabase.AlterHiveDatabaseOp op =
+        SqlAlterHiveDatabase.AlterHiveDatabaseOp.valueOf(opStr);
+    switch (op) {
+      case CHANGE_PROPS:
+        hiveDB.setParameters(newParams);
+        break;
+      case CHANGE_LOCATION:
+        hiveDB.setLocationUri(newLocation);
+        break;
+      case CHANGE_OWNER:
+        String ownerName = newParams.remove(DATABASE_OWNER_NAME);
+        String ownerType = newParams.remove(DATABASE_OWNER_TYPE);
+        hiveDB.setOwnerName(ownerName);
+        switch (ownerType) {
+          case SqlAlterHiveDatabaseOwner.ROLE_OWNER:
+            hiveDB.setOwnerType(PrincipalType.ROLE);
+            break;
+          case SqlAlterHiveDatabaseOwner.USER_OWNER:
+            hiveDB.setOwnerType(PrincipalType.USER);
+            break;
+          default:
+            throw new CatalogException("Unsupported database owner type: " + 
ownerType);
+        }
+        break;
+      default:
+        throw new CatalogException("Unsupported alter database op:" + opStr);
+    }
+    // is_generic is deprecated, remove it
+    if (hiveDB.getParameters() != null) {
+      hiveDB.getParameters().remove(CatalogPropertiesUtil.IS_GENERIC);
+    }
+    return hiveDB;
+  }
+
+  // ------ tables ------
+
+  private Table isHoodieTable(Table hiveTable) {
+    if (!hiveTable.getParameters().getOrDefault(CONNECTOR.key(), 
"").equalsIgnoreCase("hudi")
+        && !hiveTable.getParameters().getOrDefault(SPARK_SOURCE_PROVIDER, 
"").equalsIgnoreCase("hudi")) {
+      throw new HoodieCatalogException(String.format("the %s is not hoodie 
table", hiveTable.getTableName()));
+    }
+    return hiveTable;
+  }
+
+  private boolean isFlinkHoodieTable(Table hiveTable) {
+    return hiveTable.getParameters().getOrDefault(CONNECTOR.key(), 
"").equalsIgnoreCase("hudi");
+  }
+
+  @VisibleForTesting
+  public Table getHiveTable(ObjectPath tablePath) throws 
TableNotExistException {
+    try {
+      Table hiveTable = client.getTable(tablePath.getDatabaseName(), 
tablePath.getObjectName());
+      return isHoodieTable(hiveTable);
+    } catch (NoSuchObjectException e) {
+      throw new TableNotExistException(getName(), tablePath);
+    } catch (TException e) {
+      throw new HoodieCatalogException(String.format("Failed to get table %s 
from Hive metastore", tablePath.getObjectName()));
+    }
+  }
+
+  private Table translateSparkTable2Flink(ObjectPath tablePath, Table 
hiveTable) {
+    if (!isFlinkHoodieTable(hiveTable)) {
+      try {
+        Map<String, String> parameters = hiveTable.getParameters();
+        
parameters.putAll(TableOptionProperties.translateSparkTableProperties2Flink(hiveTable));
+        String path = hiveTable.getSd().getLocation();
+        parameters.put(PATH.key(), path);
+        if 
(!parameters.containsKey(FlinkOptions.HIVE_STYLE_PARTITIONING.key())) {
+          Path hoodieTablePath = new Path(path);
+          boolean hiveStyle = Arrays.stream(FSUtils.getFs(hoodieTablePath, 
hiveConf).listStatus(hoodieTablePath))
+              .map(fileStatus -> fileStatus.getPath().getName())
+              .filter(f -> !f.equals(".hoodie") && !f.equals("default"))
+              .anyMatch(FilePathUtils::isHiveStylePartitioning);
+          parameters.put(FlinkOptions.HIVE_STYLE_PARTITIONING.key(), 
String.valueOf(hiveStyle));
+        }
+        client.alter_table(tablePath.getDatabaseName(), 
tablePath.getObjectName(), hiveTable);
+      } catch (Exception e) {
+        throw new HoodieCatalogException("Failed to update table schema", e);
+      }
+    }
+    return hiveTable;
+  }
+
+  @Override
+  public CatalogBaseTable getTable(ObjectPath tablePath) throws 
TableNotExistException, CatalogException {
+    checkNotNull(tablePath, "tablePath cannot be null");
+    Table hiveTable = getHiveTable(tablePath);
+    hiveTable = translateSparkTable2Flink(tablePath, hiveTable);
+    String path = hiveTable.getSd().getLocation();
+    Map<String, String> parameters = hiveTable.getParameters();
+    Schema latestTableSchema = StreamerUtil.getLatestTableSchema(path, 
hiveConf);
+    org.apache.flink.table.api.Schema schema;
+    if (latestTableSchema != null) {
+      org.apache.flink.table.api.Schema.Builder builder = 
org.apache.flink.table.api.Schema.newBuilder()
+          
.fromRowDataType(AvroSchemaConverter.convertToDataType(latestTableSchema));
+      String pkConstraintName = parameters.get(PK_CONSTRAINT_NAME);
+      if (!StringUtils.isNullOrEmpty(pkConstraintName)) {
+        builder.primaryKeyNamed(pkConstraintName, 
StringUtils.split(parameters.get(PK_COLUMNS), ","));
+      }
+      schema = builder.build();
+    } else {
+      LOG.warn("{} does not have any hoodie schema, and use hive table schema 
to infer the table schema", tablePath);
+      schema = HiveSchemaUtils.convertTableSchema(hiveTable);
+    }
+    return CatalogTable.of(schema, parameters.get(COMMENT),
+        HiveSchemaUtils.getFieldNames(hiveTable.getPartitionKeys()), 
parameters);
+  }
+
+  @Override
+  public void createTable(ObjectPath tablePath, CatalogBaseTable table, 
boolean ignoreIfExists)
+      throws TableAlreadyExistException, DatabaseNotExistException, 
CatalogException {
+    checkNotNull(tablePath, "tablePath cannot be null");
+    checkNotNull(table, "table cannot be null");
+
+    if (!databaseExists(tablePath.getDatabaseName())) {
+      throw new DatabaseNotExistException(getName(), 
tablePath.getDatabaseName());
+    }
+
+    if (!table.getOptions().getOrDefault(CONNECTOR.key(), 
"").equalsIgnoreCase("hudi")) {
+      throw new HoodieCatalogException(String.format("The %s is not hoodie 
table", tablePath.getObjectName()));
+    }
+
+    if (table instanceof CatalogView) {
+      throw new HoodieCatalogException("CREATE VIEW is not supported.");
+    }
+
+    try {
+      boolean isMorTable = OptionsResolver.isMorTable(table.getOptions());
+      Table hiveTable = instantiateHiveTable(tablePath, table, 
inferTablePath(tablePath, table), isMorTable);
+      //create hive table
+      client.createTable(hiveTable);
+      //init hoodie metaClient
+      initTableIfNotExists(tablePath, (CatalogTable)table);
+    } catch (AlreadyExistsException e) {
+      if (!ignoreIfExists) {
+        throw new TableAlreadyExistException(getName(), tablePath, e);
+      }
+    } catch (Exception e) {
+      throw new HoodieCatalogException(
+          String.format("Failed to create table %s", tablePath.getFullName()), 
e);
+    }
+  }
+
+  private void initTableIfNotExists(ObjectPath tablePath, CatalogTable 
catalogTable) {
+    Configuration flinkConf = 
Configuration.fromMap(applyOptionsHook(catalogTable.getOptions()));
+    final String avroSchema = 
AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType()).toString();
+    flinkConf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, avroSchema);
+
+    // stores two copies of options:
+    // - partition keys
+    // - primary keys
+    // because the HoodieTableMetaClient is a heavy impl, we try to avoid 
initializing it
+    // when calling #getTable.
+
+    if (catalogTable.getUnresolvedSchema().getPrimaryKey().isPresent()) {
+      final String pkColumns = String.join(",", 
catalogTable.getUnresolvedSchema().getPrimaryKey().get().getColumnNames());
+      String recordKey = flinkConf.get(FlinkOptions.RECORD_KEY_FIELD);
+      if (!Objects.equals(pkColumns, recordKey)) {
+        throw new HoodieCatalogException(String.format("%s and %s are the 
different", pkColumns, recordKey));
+      }
+    }
+
+    if (catalogTable.isPartitioned()) {
+      final String partitions = String.join(",", 
catalogTable.getPartitionKeys());
+      flinkConf.setString(FlinkOptions.PARTITION_PATH_FIELD, partitions);
+    }
+
+    if (!flinkConf.getOptional(PATH).isPresent()) {
+      flinkConf.setString(PATH, inferTablePath(tablePath, catalogTable));
+    }
+
+    flinkConf.setString(FlinkOptions.TABLE_NAME, tablePath.getObjectName());
+    try {
+      StreamerUtil.initTableIfNotExists(flinkConf);
+    } catch (IOException e) {
+      throw new HoodieCatalogException("Initialize table exception.", e);
+    }
+  }
+
+  private String inferTablePath(ObjectPath tablePath, CatalogBaseTable table) {
+    String location = table.getOptions().getOrDefault(PATH.key(), "");
+    if (StringUtils.isNullOrEmpty(location)) {
+      try {
+        Path dbLocation = new 
Path(client.getDatabase(tablePath.getDatabaseName()).getLocationUri());
+        location = new Path(dbLocation, tablePath.getObjectName()).toString();
+      } catch (TException e) {
+        throw new HoodieCatalogException(String.format("Failed to infer hoodie 
table path for table %s", tablePath), e);
+      }
+    }
+    return location;
+  }
+
+  private Map<String, String> applyOptionsHook(Map<String, String> options) {
+    Map<String, String> properties = new HashMap<>(options);
+    if (!options.containsKey(FlinkOptions.RECORD_KEY_FIELD.key())) {
+      properties.put(FlinkOptions.RECORD_KEY_FIELD.key(), 
FlinkOptions.RECORD_KEY_FIELD.defaultValue());
+    }
+    if (!options.containsKey(FlinkOptions.PRECOMBINE_FIELD.key())) {
+      properties.put(FlinkOptions.PRECOMBINE_FIELD.key(), 
FlinkOptions.PRECOMBINE_FIELD.defaultValue());
+    }
+    if (!options.containsKey(FlinkOptions.TABLE_TYPE.key())) {
+      properties.put(FlinkOptions.TABLE_TYPE.key(), 
FlinkOptions.TABLE_TYPE.defaultValue());
+    }
+    return properties;
+  }
+
+  private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable 
table, String location, boolean useRealTimeInputFormat) throws IOException {
+    // let Hive set default parameters for us, e.g. serialization.format
+    Table hiveTable =
+        org.apache.hadoop.hive.ql.metadata.Table.getEmptyTable(
+            tablePath.getDatabaseName(), tablePath.getObjectName());
+
+    if 
(Boolean.parseBoolean(table.getOptions().get(CatalogOptions.HIVE_IS_EXTERNAL.key())))
 {
+      hiveTable.setTableType(TableType.EXTERNAL_TABLE.toString());
+    }
+
+    hiveTable.setOwner(UserGroupInformation.getCurrentUser().getUserName());
+    hiveTable.setCreateTime((int) (System.currentTimeMillis() / 1000));
+
+    Map<String, String> properties = applyOptionsHook(table.getOptions());
+
+    // Table comment
+    if (table.getComment() != null) {
+      properties.put(COMMENT, table.getComment());
+    }
+
+    //set pk
+    if (table.getUnresolvedSchema().getPrimaryKey().isPresent()) {
+      String pkColumns = String.join(",", 
table.getUnresolvedSchema().getPrimaryKey().get().getColumnNames());
+      String recordKey = 
properties.getOrDefault(FlinkOptions.RECORD_KEY_FIELD.key(), 
FlinkOptions.RECORD_KEY_FIELD.defaultValue());
+      if (!Objects.equals(pkColumns, recordKey)) {
+        throw new HoodieCatalogException(
+            String.format("If the table has primaryKey, the primaryKey should 
be the the same as the recordKey, but pk %s and recordKey %s are the different",
+                pkColumns,
+                recordKey));
+      }
+      properties.put(PK_CONSTRAINT_NAME, 
table.getUnresolvedSchema().getPrimaryKey().get().getConstraintName());
+      properties.put(PK_COLUMNS, pkColumns);
+    }
+
+    if (!properties.containsKey(FlinkOptions.PATH.key())) {
+      properties.put(FlinkOptions.PATH.key(), location);
+    }
+
+    //set sd
+    StorageDescriptor sd = new StorageDescriptor();
+    List<FieldSchema> allColumns = 
HiveSchemaUtils.createHiveColumns(table.getSchema());
+
+    // Table columns and partition keys
+    if (table instanceof CatalogTable) {
+      CatalogTable catalogTable = (CatalogTable) table;
+
+      if (catalogTable.isPartitioned()) {
+        int partitionKeySize = catalogTable.getPartitionKeys().size();
+        List<FieldSchema> regularColumns =
+            allColumns.subList(0, allColumns.size() - partitionKeySize);
+        List<FieldSchema> partitionColumns =
+            allColumns.subList(
+                allColumns.size() - partitionKeySize, allColumns.size());
+
+        sd.setCols(regularColumns);
+        hiveTable.setPartitionKeys(partitionColumns);
+      } else {
+        sd.setCols(allColumns);
+        hiveTable.setPartitionKeys(new ArrayList<>());
+      }
+    } else {
+      sd.setCols(allColumns);
+    }
+
+    HoodieFileFormat baseFileFormat = HoodieFileFormat.PARQUET;
+    //ignore uber input Format
+    String inputFormatClassName = 
HoodieInputFormatUtils.getInputFormatClassName(baseFileFormat, 
useRealTimeInputFormat);
+    String outputFormatClassName = 
HoodieInputFormatUtils.getOutputFormatClassName(baseFileFormat);
+    String serDeClassName = 
HoodieInputFormatUtils.getSerDeClassName(baseFileFormat);
+    sd.setInputFormat(inputFormatClassName);
+    sd.setOutputFormat(outputFormatClassName);
+    Map<String, String> serdeProperties = new HashMap<>();
+    serdeProperties.put("path", location);
+    serdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, 
String.valueOf(!useRealTimeInputFormat));
+    serdeProperties.put("serialization.format", "1");
+
+    
serdeProperties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark((CatalogTable)table,
 hiveConf, properties));
+
+    sd.setSerdeInfo(new SerDeInfo(null, serDeClassName, serdeProperties));
+
+    sd.setLocation(location);
+
+    hiveTable.setParameters(properties);
+    return hiveTable;
+  }
+
+  @Override
+  public List<String> listTables(String databaseName)
+      throws DatabaseNotExistException, CatalogException {
+    checkArgument(
+        !isNullOrWhitespaceOnly(databaseName), "databaseName cannot be null or 
empty");
+
+    try {
+      return client.getAllTables(databaseName);
+    } catch (UnknownDBException e) {
+      throw new DatabaseNotExistException(getName(), databaseName);
+    } catch (TException e) {
+      throw new HoodieCatalogException(
+          String.format("Failed to list tables in database %s", databaseName), 
e);
+    }
+  }
+
+  @Override
+  public List<String> listViews(String databaseName)
+      throws DatabaseNotExistException, CatalogException {
+    throw new HoodieCatalogException("Hoodie catalog does not support to 
listViews");
+  }
+
+  @Override
+  public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+    checkNotNull(tablePath, "tablePath cannot be null");
+
+    try {
+      return client.tableExists(tablePath.getDatabaseName(), 
tablePath.getObjectName());
+    } catch (UnknownDBException e) {
+      return false;
+    } catch (TException e) {
+      throw new CatalogException(
+          String.format(
+              "Failed to check whether table %s exists or not.",
+              tablePath.getFullName()),
+          e);
+    }
+  }
+
+  @Override
+  public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    checkNotNull(tablePath, "tablePath cannot be null");
+
+    try {

Review Comment:
   yes drop hive&hdfs table



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to