[HUDI-3106] Fix HiveSyncTool not sync schema (#4452)
This commit is contained in:
@@ -18,6 +18,11 @@
|
||||
|
||||
package org.apache.hudi.hive;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -31,12 +36,6 @@ import org.apache.hudi.hive.util.Parquet2SparkSchemaUtils;
|
||||
import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent;
|
||||
import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType;
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.schema.GroupType;
|
||||
@@ -168,26 +167,22 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
// check if isDropPartition
|
||||
boolean isDropPartition = hoodieHiveClient.isDropPartition();
|
||||
|
||||
// check if schemaChanged
|
||||
boolean schemaChanged = false;
|
||||
// Get the parquet schema for this table looking at the latest commit
|
||||
MessageType schema = hoodieHiveClient.getDataSchema();
|
||||
|
||||
if (!isDropPartition) {
|
||||
// Get the parquet schema for this table looking at the latest commit
|
||||
MessageType schema = hoodieHiveClient.getDataSchema();
|
||||
|
||||
// Currently HoodieBootstrapRelation does support reading bootstrap MOR rt table,
|
||||
// so we disable the syncAsSparkDataSourceTable here to avoid read such kind table
|
||||
// by the data source way (which will use the HoodieBootstrapRelation).
|
||||
// TODO after we support bootstrap MOR rt table in HoodieBootstrapRelation[HUDI-2071], we can remove this logical.
|
||||
if (hoodieHiveClient.isBootstrap()
|
||||
&& hoodieHiveClient.getTableType() == HoodieTableType.MERGE_ON_READ
|
||||
&& !readAsOptimized) {
|
||||
cfg.syncAsSparkDataSourceTable = false;
|
||||
}
|
||||
// Sync schema if needed
|
||||
schemaChanged = syncSchema(tableName, tableExists, useRealtimeInputFormat, readAsOptimized, schema);
|
||||
// Currently HoodieBootstrapRelation does support reading bootstrap MOR rt table,
|
||||
// so we disable the syncAsSparkDataSourceTable here to avoid read such kind table
|
||||
// by the data source way (which will use the HoodieBootstrapRelation).
|
||||
// TODO after we support bootstrap MOR rt table in HoodieBootstrapRelation[HUDI-2071], we can remove this logical.
|
||||
if (hoodieHiveClient.isBootstrap()
|
||||
&& hoodieHiveClient.getTableType() == HoodieTableType.MERGE_ON_READ
|
||||
&& !readAsOptimized) {
|
||||
cfg.syncAsSparkDataSourceTable = false;
|
||||
}
|
||||
|
||||
// Sync schema if needed
|
||||
boolean schemaChanged = syncSchema(tableName, tableExists, useRealtimeInputFormat, readAsOptimized, schema);
|
||||
|
||||
LOG.info("Schema sync complete. Syncing partitions for " + tableName);
|
||||
// Get the last time we successfully synced partitions
|
||||
Option<String> lastCommitTimeSynced = Option.empty();
|
||||
|
||||
Reference in New Issue
Block a user