You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/07/10 06:22:57 UTC

[GitHub] [hudi] xiaozhch5 commented on a diff in pull request #6013: [HUDI-4089] Support HMS for flink HoodieCatalog

xiaozhch5 commented on code in PR #6013:
URL: https://github.com/apache/hudi/pull/6013#discussion_r917345825


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,907 @@
+/*
+ * 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.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.TableSchemaResolver;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.configuration.FlinkOptions;
+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.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.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.HoodieCatalogFactoryOptions.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 final boolean iniTable;
+  private IMetaStoreClient client;
+
+  public HoodieHiveCatalog(String catalogName, String defaultDatabase, String hiveConf, boolean iniTable) {
+    this(catalogName, defaultDatabase, HoodieCatalogUtil.createHiveConf(hiveConf), iniTable, false);
+  }
+
+  public HoodieHiveCatalog(String catalogName, String defaultDatabase, HiveConf hiveConf, boolean iniTable, boolean allowEmbedded) {
+    super(catalogName, defaultDatabase == null ? DEFAULT_DB : defaultDatabase);
+    this.hiveConf = hiveConf;
+    this.iniTable = iniTable;
+    if (!allowEmbedded) {
+      checkArgument(
+          !HoodieCatalogUtil.isEmbeddedMetastore(this.hiveConf),
+          "Embedded metastore is not allowed. Make sure you have set a valid value for "
+              + HiveConf.ConfVars.METASTOREURIS.toString());
+    }
+    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 HiveMetaStoreClient", e);
+      }
+      LOG.info("Connected to Hive metastore");
+    }
+    if (!databaseExists(getDefaultDatabase())) {
+      throw new HoodieCatalogException(
+          String.format(
+              "Configured default database %s doesn't exist in catalog %s.",
+              getDefaultDatabase(), getName()));
+    }
+  }
+
+  @Override
+  public void close() throws CatalogException {
+    if (client != null) {
+      client.close();
+      client = null;
+      LOG.info("Close connection 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);
+    }
+  }
+
+  public 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 checkHoodieTable(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 isFlinkTable(Table hiveTable) {
+    return hiveTable.getParameters().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi");
+  }
+
+  private org.apache.avro.Schema getLatestTableSchema(String path) {
+    if (StringUtils.isNullOrEmpty(path) || !StreamerUtil.tableExists(path, hiveConf)) {
+      return null;
+    }
+
+    try {
+      HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(path, hiveConf);
+      return new TableSchemaResolver(metaClient).getTableAvroSchema(false);
+    } catch (Exception e) {
+      LOG.warn("Error while resolving the latest table schema", e);
+    }
+    return null;
+  }
+
+  @VisibleForTesting
+  public Table getHiveTable(ObjectPath tablePath) throws TableNotExistException {
+    try {
+      Table hiveTable = client.getTable(tablePath.getDatabaseName(), tablePath.getObjectName());
+      return checkHoodieTable(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 (!isFlinkTable(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 = getLatestTableSchema(path);
+    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 = TableOptionProperties.convertTableSchema(hiveTable);
+    }
+    return CatalogTable.of(schema, parameters.get(COMMENT),
+        TableOptionProperties.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("Hoodie catalog does not support to CREATE VIEW.");
+    }
+
+    try {
+      boolean isMorTable = table.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(),
+          FlinkOptions.TABLE_TYPE.defaultValue()).equalsIgnoreCase(FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
+      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) {
+    if (!iniTable) {
+      LOG.info("Skip init table.");
+      return;
+    }
+    Configuration flinkConf = Configuration.fromMap(applyOptionsHook(catalogTable.getOptions()));
+    flinkConf.addAllToProperties(hiveConf.getAllProperties());
+    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());
+    hiveTable.setOwner(UserGroupInformation.getCurrentUser().getUserName());
+    hiveTable.setCreateTime((int) (System.currentTimeMillis() / 1000));
+
+    Map<String, String> properties = applyOptionsHook(table.getOptions());
+    properties.put("EXTERNAL", "TRUE");

Review Comment:
   If it's better for user to customize the table type of "EXTERNAL" or "MANAGED"?
   
   Using flink engine and hudi catalog, if we drop the table, the hdfs dir will be deleted.
   
   Using spark engine, if we set 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog', spark create the "MANAGED" table by default in HMS. And if we drop the table, the hdfs dir will be deleted.
   
   I think if we can  customize the table type of "EXTERNAL" or "MANAGED", we can control the hdfs dir will be deleted or not.
   @danny0405 



-- 
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-unsubscribe@hudi.apache.org

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