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/12 08:53:29 UTC

[GitHub] [hudi] danny0405 commented on a diff in pull request #6082: [HUDI-4098]Support HMS for flink HudiCatalog

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java:
##########
@@ -52,21 +54,35 @@ public Catalog createCatalog(Context context) {
         FactoryUtil.createCatalogFactoryHelper(this, context);
     helper.validate();
 
-    return new HoodieCatalog(
-        context.getName(),
-        (Configuration) helper.getOptions());
+    if (helper.getOptions().get(HoodieCatalogFactoryOptions.MODE).equalsIgnoreCase("hms")) {
+      return new HoodieHiveCatalog(
+          context.getName(),
+          helper.getOptions().get(HoodieCatalogFactoryOptions.DEFAULT_DATABASE),
+          helper.getOptions().get(HoodieCatalogFactoryOptions.HIVE_CONF_DIR),
+          helper.getOptions().get(HoodieCatalogFactoryOptions.INIT_FS_TABLE));
+    } else if (helper.getOptions().get(HoodieCatalogFactoryOptions.MODE).equalsIgnoreCase("dfs")) {
+      return new HoodieCatalog(
+          context.getName(),
+          (Configuration) helper.getOptions());
+    } else {
+      throw new HoodieCatalogException("hoodie catalog supports only the hms and dfs modes.");
+    }

Review Comment:
   `hoodie catalog supports only the hms and dfs modes.` -> `Invalid catalog mode: {}, supported modes: [hms, dfs]`.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.table.catalog.CommonCatalogOptions;
+
+/** {@link ConfigOption}s for {@link HoodieHiveCatalog}. */
+public class HoodieCatalogFactoryOptions {
+  public static final String DEFAULT_DB = "default";
+  public static final String HIVE_SITE_FILE = "hive-site.xml";
+
+  public static final ConfigOption<String> DEFAULT_DATABASE =
+      ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY)
+          .stringType()
+          .defaultValue(DEFAULT_DB);
+
+  public static final ConfigOption<String> HIVE_CONF_DIR =
+      ConfigOptions.key("hive-conf-dir").stringType().noDefaultValue();
+

Review Comment:
   `hive-conf-dir` -> `hive.conf.dir`



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java:
##########
@@ -52,21 +54,35 @@ public Catalog createCatalog(Context context) {
         FactoryUtil.createCatalogFactoryHelper(this, context);
     helper.validate();
 
-    return new HoodieCatalog(
-        context.getName(),
-        (Configuration) helper.getOptions());
+    if (helper.getOptions().get(HoodieCatalogFactoryOptions.MODE).equalsIgnoreCase("hms")) {
+      return new HoodieHiveCatalog(

Review Comment:
   would suggest to use the `switch case` clause.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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);
+      }

Review Comment:
   `HiveMetaStoreClient` -> `hive metastore client`



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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());
+    }

Review Comment:
   `toString()` can be avoided.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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 {

Review Comment:
   public -> private ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.table.catalog.CommonCatalogOptions;
+
+/** {@link ConfigOption}s for {@link HoodieHiveCatalog}. */
+public class HoodieCatalogFactoryOptions {
+  public static final String DEFAULT_DB = "default";
+  public static final String HIVE_SITE_FILE = "hive-site.xml";

Review Comment:
   Can we merge this clazz into `CatalogOptions`.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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.");
+    }

Review Comment:
   CREATE VIEW is not supported.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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);

Review Comment:
   `OptionsResolver.isMorTable`



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java:
##########
@@ -128,4 +173,179 @@ public static Map<String, String> getTableOptions(Map<String, String> options) {
     NON_OPTION_KEYS.forEach(copied::remove);
     return copied;
   }
+
+  public static Map<String, String> translateFlinkTableProperties2Spark(CatalogTable catalogTable, Configuration hadoopConf) {
+    Schema schema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType());
+    MessageType messageType = TableSchemaResolver.convertAvroSchemaToParquet(schema, hadoopConf);
+    String sparkVersion = catalogTable.getOptions().getOrDefault(SPARK_VERSION, DEFAULT_SPARK_VERSION);
+    return SparkDataSourceTableUtils.getSparkTableProperties(catalogTable.getPartitionKeys(), sparkVersion, 4000, messageType);

Review Comment:
   Do we also need to add the options in KEY_MAPPING ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.table.catalog.CommonCatalogOptions;
+
+/** {@link ConfigOption}s for {@link HoodieHiveCatalog}. */
+public class HoodieCatalogFactoryOptions {
+  public static final String DEFAULT_DB = "default";
+  public static final String HIVE_SITE_FILE = "hive-site.xml";
+
+  public static final ConfigOption<String> DEFAULT_DATABASE =
+      ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY)
+          .stringType()
+          .defaultValue(DEFAULT_DB);
+
+  public static final ConfigOption<String> HIVE_CONF_DIR =
+      ConfigOptions.key("hive-conf-dir").stringType().noDefaultValue();
+
+  public static final ConfigOption<String> MODE =
+      ConfigOptions.key("mode").stringType().defaultValue("dfs");
+
+  public static final ConfigOption<Boolean> INIT_FS_TABLE =
+      ConfigOptions.key("init.fs.table").booleanType().defaultValue(true);

Review Comment:
   `INIT_FS_TABLE` option should be avoided, either we always create the table dir path, either we ignore it.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java:
##########
@@ -54,6 +57,45 @@ public static org.apache.hadoop.conf.Configuration getHadoopConf(Configuration c
     return hadoopConf;
   }
 
+  /**
+   * Returns a new hadoop configuration that is initialized with the given hadoopConfDir.
+   *
+   * @param hadoopConfDir Hadoop conf directory path.
+   * @return A Hadoop configuration instance.
+   */
+  public static org.apache.hadoop.conf.Configuration getHadoopConfiguration(String hadoopConfDir) {
+    if (new File(hadoopConfDir).exists()) {
+      List<File> possiableConfFiles = new ArrayList<File>();

Review Comment:
   We may need to consider whether to expose the explicit hadoop config dir in this version, different with hive, people always configure their hadoop conf dir through their env variables, from this point, i would suggest to use the `HadoopConfigurations.getHadoopConf` directly, which is enough for most cases.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactoryOptions.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.table.catalog.CommonCatalogOptions;
+
+/** {@link ConfigOption}s for {@link HoodieHiveCatalog}. */
+public class HoodieCatalogFactoryOptions {
+  public static final String DEFAULT_DB = "default";
+  public static final String HIVE_SITE_FILE = "hive-site.xml";
+
+  public static final ConfigOption<String> DEFAULT_DATABASE =
+      ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY)
+          .stringType()
+          .defaultValue(DEFAULT_DB);
+
+  public static final ConfigOption<String> HIVE_CONF_DIR =
+      ConfigOptions.key("hive-conf-dir").stringType().noDefaultValue();
+
+  public static final ConfigOption<String> MODE =
+      ConfigOptions.key("mode").stringType().defaultValue("dfs");
+
+  public static final ConfigOption<Boolean> INIT_FS_TABLE =
+      ConfigOptions.key("init.fs.table").booleanType().defaultValue(true);

Review Comment:
   Can we give some explanation for these options ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.configuration.HadoopConfigurations;
+
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+import static org.apache.hudi.table.catalog.HoodieCatalogFactoryOptions.HIVE_SITE_FILE;
+
+/**
+ * Utilities for Hoodie Catalog.
+ */
+public class HoodieCatalogUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(HoodieCatalogUtil.class);
+
+  /**
+   * Returns a new {@code HiveConf}.
+   *
+   * @param hiveConfDir Hive conf directory path.
+   * @return A HiveConf instance.
+   */
+  public static HiveConf createHiveConf(@Nullable String hiveConfDir) {
+    // create HiveConf from hadoop configuration with hadoop conf directory configured.
+    Configuration hadoopConf = HadoopConfigurations.getHadoopConfiguration(hiveConfDir);
+    if (isNullOrWhitespaceOnly(hiveConfDir) || hadoopConf == null) {

Review Comment:
   Why not just use `HadoopConfigurations.getHadoopConf(Configuration conf)`



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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()));

Review Comment:
   Can we create default database directly instead of throwing exception ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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();

Review Comment:
   `hiveConf.getAllProperties()` is never used, so why we add the flink conf options to it ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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");
+    }

Review Comment:
   `Close connection to` -> `Disconnect to `



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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")) {

Review Comment:
   Can `isFlinkTable(` be used here ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java:
##########
@@ -128,4 +173,179 @@ public static Map<String, String> getTableOptions(Map<String, String> options) {
     NON_OPTION_KEYS.forEach(copied::remove);
     return copied;
   }
+
+  public static Map<String, String> translateFlinkTableProperties2Spark(CatalogTable catalogTable, Configuration hadoopConf) {
+    Schema schema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType());
+    MessageType messageType = TableSchemaResolver.convertAvroSchemaToParquet(schema, hadoopConf);
+    String sparkVersion = catalogTable.getOptions().getOrDefault(SPARK_VERSION, DEFAULT_SPARK_VERSION);
+    return SparkDataSourceTableUtils.getSparkTableProperties(catalogTable.getPartitionKeys(), sparkVersion, 4000, messageType);
+  }
+
+  public static Map<String, String> translateSparkTableProperties2Flink(Map<String, String> options) {
+    if (options.containsKey(CONNECTOR.key())) {
+      return options;
+    }
+    return options.entrySet().stream().filter(e -> KEY_MAPPING.containsKey(e.getKey()))
+        .collect(Collectors.toMap(e -> KEY_MAPPING.get(e.getKey()),
+            e -> e.getKey().equalsIgnoreCase("type") ? VALUE_MAPPING.get(e.getValue()) : e.getValue()));
+  }
+
+  public static Map<String, String> translateSparkTableProperties2Flink(Table hiveTable) {
+    return translateSparkTableProperties2Flink(hiveTable.getParameters());
+  }
+
+  /** Get field names from field schemas. */
+  public static List<String> getFieldNames(List<FieldSchema> fieldSchemas) {
+    List<String> names = new ArrayList<>(fieldSchemas.size());
+    for (FieldSchema fs : fieldSchemas) {
+      names.add(fs.getName());
+    }
+    return names;
+  }
+
+  public static org.apache.flink.table.api.Schema convertTableSchema(Table hiveTable) {
+    List<FieldSchema> allCols = new ArrayList<>(hiveTable.getSd().getCols());
+    allCols.addAll(hiveTable.getPartitionKeys());
+
+    String pkConstraintName = hiveTable.getParameters().get(PK_CONSTRAINT_NAME);
+    List<String> primaryColNames = StringUtils.isNullOrEmpty(pkConstraintName)
+        ? Collections.EMPTY_LIST
+        : StringUtils.split(hiveTable.getParameters().get(PK_COLUMNS),",");
+
+    String[] colNames = new String[allCols.size()];
+    DataType[] colTypes = new DataType[allCols.size()];
+
+    for (int i = 0; i < allCols.size(); i++) {
+      FieldSchema fs = allCols.get(i);
+
+      colNames[i] = fs.getName();
+      colTypes[i] =
+          toFlinkType(TypeInfoUtils.getTypeInfoFromTypeString(fs.getType()));
+      if (primaryColNames.contains(colNames[i])) {
+        colTypes[i] = colTypes[i].notNull();
+      }
+    }
+
+    org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder().fromFields(colNames, colTypes);
+    if (!StringUtils.isNullOrEmpty(pkConstraintName)) {
+      builder.primaryKeyNamed(pkConstraintName, primaryColNames);

Review Comment:
   Can we move all the new methods to a new tool clazz `HiveTableOptions` under package `catalog` ?



##########
hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory:
##########
@@ -15,4 +15,4 @@
 # limitations under the License.
 
 org.apache.hudi.table.HoodieTableFactory
-org.apache.hudi.table.catalog.HoodieCatalogFactory
+org.apache.hudi.table.catalog.HoodieCatalogFactory

Review Comment:
   Unnecessary change.



##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalogFactory.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CommonCatalogOptions;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.junit.jupiter.api.Test;
+
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Test cases for {@link HoodieCatalogFactory}.
+ */
+public class TestHoodieCatalogFactory {
+  private static final URL CONF_DIR =
+      Thread.currentThread().getContextClassLoader().getResource("test-catalog-factory-conf");
+
+  @Test
+  public void testCreateHiveCatalog() {
+    final String catalogName = "mycatalog";
+
+    final HoodieHiveCatalog expectedCatalog = TestHoodieCatalogUtils.createHiveCatalog(catalogName);
+
+    final Map<String, String> options = new HashMap<>();
+    options.put(CommonCatalogOptions.CATALOG_TYPE.key(), HoodieCatalogFactory.IDENTIFIER);
+    options.put(HoodieCatalogFactoryOptions.HIVE_CONF_DIR.key(), CONF_DIR.getPath());
+    options.put(HoodieCatalogFactoryOptions.MODE.key(), "hms");
+
+    final Catalog actualCatalog =

Review Comment:
   add case for dfs catalog creation.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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;

Review Comment:
   Can `StreamerUtil.getTableAvroSchema(` be used here ? Or can we move this tool to `StreamerUtil`



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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");
+    // Table comment
+    if (table.getComment() != null) {
+      properties.put(COMMENT, table.getComment());
+    }
+
+    //set sd
+    StorageDescriptor sd = new StorageDescriptor();
+    List<FieldSchema> allColumns = TableOptionProperties.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");
+    sd.setSerdeInfo(new SerDeInfo(null, serDeClassName, serdeProperties));
+
+    sd.setLocation(location);
+
+    hiveTable.setSd(sd);
+
+    properties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark((CatalogTable)table, hiveConf));
+
+    //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);
+    }
+
+    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 {
+      client.dropTable(
+          tablePath.getDatabaseName(),
+          tablePath.getObjectName(),
+          // Indicate whether associated data should be deleted.
+          // Set to 'true' for now because Flink tables shouldn't have data in Hive. Can
+          // be changed later if necessary
+          true,
+          ignoreIfNotExists);
+    } catch (NoSuchObjectException e) {
+      if (!ignoreIfNotExists) {
+        throw new TableNotExistException(getName(), tablePath);
+      }
+    } catch (TException e) {
+      throw new HoodieCatalogException(
+          String.format("Failed to drop table %s", tablePath.getFullName()), e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    checkNotNull(tablePath, "tablePath cannot be null");
+    checkArgument(
+        !isNullOrWhitespaceOnly(newTableName), "newTableName cannot be null or empty");
+
+    try {
+      // alter_table() doesn't throw a clear exception when target table doesn't exist.
+      // Thus, check the table existence explicitly
+      if (tableExists(tablePath)) {
+        ObjectPath newPath = new ObjectPath(tablePath.getDatabaseName(), newTableName);
+        // alter_table() doesn't throw a clear exception when new table already exists.
+        // Thus, check the table existence explicitly
+        if (tableExists(newPath)) {
+          throw new TableAlreadyExistException(getName(), newPath);
+        } else {
+          Table hiveTable = getHiveTable(tablePath);
+
+          //update hoodie
+          StorageDescriptor sd = hiveTable.getSd();
+          String location = sd.getLocation();
+          HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(location).setConf(hiveConf).build();
+          //Init table with new name
+          HoodieTableMetaClient.withPropertyBuilder().fromProperties(metaClient.getTableConfig().getProps())
+              .setTableName(newTableName)
+              .initTable(hiveConf, location);
+
+          hiveTable.setTableName(newTableName);
+          client.alter_table(
+              tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable);
+        }
+      } else if (!ignoreIfNotExists) {
+        throw new TableNotExistException(getName(), tablePath);
+      }
+    } catch (Exception e) {
+      throw new HoodieCatalogException(
+          String.format("Failed to rename table %s", tablePath.getFullName()), e);
+    }
+  }
+
+  @Override
+  public void alterTable(
+      ObjectPath tablePath, CatalogBaseTable newCatalogTable, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    checkNotNull(tablePath, "tablePath cannot be null");
+    checkNotNull(newCatalogTable, "newCatalogTable cannot be null");
+
+    if (!newCatalogTable.getOptions().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi")) {
+      throw new HoodieCatalogException(String.format("The %s is not hoodie table", tablePath.getObjectName()));
+    }
+    if (newCatalogTable instanceof CatalogView) {
+      throw new HoodieCatalogException("Hoodie catalog does not support to ALTER VIEW");
+    }
+
+    try {
+      Table hiveTable = getHiveTable(tablePath);
+      if (!hiveTable.getParameters().getOrDefault(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue())
+          .equalsIgnoreCase(newCatalogTable.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue()))
+          || !hiveTable.getParameters().getOrDefault(FlinkOptions.INDEX_TYPE.key(), FlinkOptions.INDEX_TYPE.defaultValue())
+          .equalsIgnoreCase(newCatalogTable.getOptions().getOrDefault(FlinkOptions.INDEX_TYPE.key(), FlinkOptions.INDEX_TYPE.defaultValue()))) {

Review Comment:
   The code is too long, maybe we can write a tool method like 
   
   ```java
   private static boolean sameOptions(Map<String, String> existingOptions, Map<String, String> newOptions, ConfigOpiton<String> key)
   ```



##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.model.HoodieTableType;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.exception.HoodieCatalogException;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Test cases for {@link HoodieHiveCatalog}.
+ */
+public class TestHoodieHiveCatalog {
+  TableSchema schema =
+      TableSchema.builder()
+          .field("uuid", DataTypes.INT().notNull())
+          .field("name", DataTypes.STRING())
+          .field("age", DataTypes.INT())
+          .field("ts", DataTypes.BIGINT())
+          .field("par1", DataTypes.STRING())
+          .primaryKey("uuid")
+          .build();
+  List<String> partitions = Collections.singletonList("par1");
+  private static HoodieHiveCatalog hoodieCatalog;
+  private final ObjectPath tablePath = new ObjectPath("default", "test");
+
+  @BeforeAll
+  public static void createCatalog() {
+    hoodieCatalog = TestHoodieCatalogUtils.createHiveCatalog();
+    hoodieCatalog.open();
+  }
+
+  @AfterEach
+  public void dropTable() throws TableNotExistException {
+    hoodieCatalog.dropTable(tablePath, true);
+  }
+
+  @AfterAll
+  public static void closeCatalog() {
+    if (hoodieCatalog != null) {
+      hoodieCatalog.close();
+    }
+  }
+
+  @ParameterizedTest
+  @EnumSource(value = HoodieTableType.class)
+  public void testCreateAndGetHoodieTable(HoodieTableType tableType) throws Exception {
+    Map<String, String> originOptions = new HashMap<>();
+    originOptions.put(FactoryUtil.CONNECTOR.key(), "hudi");
+    originOptions.put(FlinkOptions.TABLE_TYPE.key(), tableType.toString());
+
+    CatalogTable table =
+        new CatalogTableImpl(schema, partitions, originOptions, "hudi table");
+    hoodieCatalog.createTable(tablePath, table, false);
+
+    CatalogBaseTable table1 = hoodieCatalog.getTable(tablePath);
+    assertEquals(table1.getOptions().get(CONNECTOR.key()), "hudi");
+    assertEquals(table1.getOptions().get(FlinkOptions.TABLE_TYPE.key()), tableType.toString());
+    assertEquals(table1.getOptions().get(FlinkOptions.RECORD_KEY_FIELD.key()), "uuid");
+    assertEquals(table1.getOptions().get(FlinkOptions.PRECOMBINE_FIELD.key()), "ts");
+    assertEquals(table1.getUnresolvedSchema().getPrimaryKey().get().getColumnNames(), Collections.singletonList("uuid"));
+    assertEquals(((CatalogTable)table1).getPartitionKeys(), Collections.singletonList("par1"));
+  }
+
+  @Test
+  public void testCreateNonHoodieTable() throws TableAlreadyExistException, DatabaseNotExistException {
+    CatalogTable table =
+        new CatalogTableImpl(schema, Collections.emptyMap(), "hudi table");
+    try {
+      hoodieCatalog.createTable(tablePath, table, false);
+    } catch (HoodieCatalogException e) {
+      assertEquals(String.format("The %s is not hoodie table", tablePath.getObjectName()), e.getMessage());
+    }

Review Comment:
   Do we have plan to support non-hoodie table creation ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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");
+    // Table comment

Review Comment:
   Maybe we should keep sync with spark catalog, always create managed table.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -0,0 +1,908 @@
+/*
+ * 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.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.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");
+    // Table comment
+    if (table.getComment() != null) {
+      properties.put(COMMENT, table.getComment());
+    }
+
+    //set sd
+    StorageDescriptor sd = new StorageDescriptor();
+    List<FieldSchema> allColumns = TableOptionProperties.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");
+    sd.setSerdeInfo(new SerDeInfo(null, serDeClassName, serdeProperties));
+
+    sd.setLocation(location);
+
+    hiveTable.setSd(sd);
+
+    properties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark((CatalogTable)table, hiveConf));
+
+    //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);
+    }
+
+    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 {
+      client.dropTable(
+          tablePath.getDatabaseName(),
+          tablePath.getObjectName(),
+          // Indicate whether associated data should be deleted.
+          // Set to 'true' for now because Flink tables shouldn't have data in Hive. Can
+          // be changed later if necessary
+          true,
+          ignoreIfNotExists);
+    } catch (NoSuchObjectException e) {
+      if (!ignoreIfNotExists) {
+        throw new TableNotExistException(getName(), tablePath);
+      }
+    } catch (TException e) {
+      throw new HoodieCatalogException(
+          String.format("Failed to drop table %s", tablePath.getFullName()), e);
+    }
+  }
+
+  @Override
+  public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+      throws TableNotExistException, TableAlreadyExistException, CatalogException {
+    checkNotNull(tablePath, "tablePath cannot be null");
+    checkArgument(
+        !isNullOrWhitespaceOnly(newTableName), "newTableName cannot be null or empty");
+
+    try {
+      // alter_table() doesn't throw a clear exception when target table doesn't exist.
+      // Thus, check the table existence explicitly
+      if (tableExists(tablePath)) {
+        ObjectPath newPath = new ObjectPath(tablePath.getDatabaseName(), newTableName);
+        // alter_table() doesn't throw a clear exception when new table already exists.
+        // Thus, check the table existence explicitly
+        if (tableExists(newPath)) {
+          throw new TableAlreadyExistException(getName(), newPath);
+        } else {
+          Table hiveTable = getHiveTable(tablePath);
+
+          //update hoodie
+          StorageDescriptor sd = hiveTable.getSd();
+          String location = sd.getLocation();
+          HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(location).setConf(hiveConf).build();
+          //Init table with new name
+          HoodieTableMetaClient.withPropertyBuilder().fromProperties(metaClient.getTableConfig().getProps())
+              .setTableName(newTableName)
+              .initTable(hiveConf, location);
+
+          hiveTable.setTableName(newTableName);
+          client.alter_table(
+              tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable);
+        }
+      } else if (!ignoreIfNotExists) {
+        throw new TableNotExistException(getName(), tablePath);
+      }
+    } catch (Exception e) {
+      throw new HoodieCatalogException(
+          String.format("Failed to rename table %s", tablePath.getFullName()), e);
+    }
+  }
+
+  @Override
+  public void alterTable(
+      ObjectPath tablePath, CatalogBaseTable newCatalogTable, boolean ignoreIfNotExists)
+      throws TableNotExistException, CatalogException {
+    checkNotNull(tablePath, "tablePath cannot be null");
+    checkNotNull(newCatalogTable, "newCatalogTable cannot be null");
+
+    if (!newCatalogTable.getOptions().getOrDefault(CONNECTOR.key(), "").equalsIgnoreCase("hudi")) {
+      throw new HoodieCatalogException(String.format("The %s is not hoodie table", tablePath.getObjectName()));
+    }
+    if (newCatalogTable instanceof CatalogView) {
+      throw new HoodieCatalogException("Hoodie catalog does not support to ALTER VIEW");
+    }
+
+    try {
+      Table hiveTable = getHiveTable(tablePath);
+      if (!hiveTable.getParameters().getOrDefault(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue())
+          .equalsIgnoreCase(newCatalogTable.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE.defaultValue()))
+          || !hiveTable.getParameters().getOrDefault(FlinkOptions.INDEX_TYPE.key(), FlinkOptions.INDEX_TYPE.defaultValue())
+          .equalsIgnoreCase(newCatalogTable.getOptions().getOrDefault(FlinkOptions.INDEX_TYPE.key(), FlinkOptions.INDEX_TYPE.defaultValue()))) {
+        throw new HoodieCatalogException("Hoodie catalog does not support to alter table type and index type");
+      }
+    } catch (TableNotExistException e) {
+      if (!ignoreIfNotExists) {
+        throw e;
+      }
+      return;
+    }
+
+    try {
+      boolean isMorTable = newCatalogTable.getOptions().getOrDefault(FlinkOptions.TABLE_TYPE.key(),
+          FlinkOptions.TABLE_TYPE.defaultValue()).equalsIgnoreCase(FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
+      Table hiveTable = instantiateHiveTable(tablePath, newCatalogTable, inferTablePath(tablePath, newCatalogTable), isMorTable);

Review Comment:
   `OptionsResolver.isMorTable`



-- 
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