bvaradar commented on a change in pull request #1687:
URL: https://github.com/apache/hudi/pull/1687#discussion_r444684726
##########
File path: hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java
##########
@@ -146,21 +146,22 @@ private void syncSchema(String tableName, boolean
tableExists, boolean useRealTi
// Check and sync schema
if (!tableExists) {
LOG.info("Hive table " + tableName + " is not found. Creating it");
- if (!useRealTimeInputFormat) {
- String inputFormatClassName = cfg.usePreApacheInputFormat ?
com.uber.hoodie.hadoop.HoodieInputFormat.class.getName()
- : HoodieParquetInputFormat.class.getName();
- hoodieHiveClient.createTable(tableName, schema, inputFormatClassName,
MapredParquetOutputFormat.class.getName(),
- ParquetHiveSerDe.class.getName());
- } else {
- // Custom serde will not work with ALTER TABLE REPLACE COLUMNS
- //
https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive
- // /ql/exec/DDLTask.java#L3488
- String inputFormatClassName =
- cfg.usePreApacheInputFormat ?
com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName()
- : HoodieParquetRealtimeInputFormat.class.getName();
- hoodieHiveClient.createTable(tableName, schema, inputFormatClassName,
MapredParquetOutputFormat.class.getName(),
- ParquetHiveSerDe.class.getName());
+ HoodieFileFormat baseFileFormat =
HoodieFileFormat.valueOf(cfg.baseFileFormat.toUpperCase());
+ String inputFormatClassName =
HoodieInputFormatUtils.getInputFormatClassName(baseFileFormat,
useRealTimeInputFormat,
+ new Configuration());
+
+ if (baseFileFormat.equals(HoodieFileFormat.PARQUET) &&
cfg.usePreApacheInputFormat) {
+ // Parquet input format had an InputFormat class visible under the old
naming scheme.
+ inputFormatClassName = useRealTimeInputFormat
+ ?
com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName()
+ : com.uber.hoodie.hadoop.HoodieInputFormat.class.getName();
}
+
+ // Custom serde will not work with ALTER TABLE REPLACE COLUMNS
+ //
https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive
+ // /ql/exec/DDLTask.java#L3488
+ hoodieHiveClient.createTable(tableName, schema, inputFormatClassName,
MapredParquetOutputFormat.class.getName(),
Review comment:
I did not mean to implement output formats. For Parquet, the
outputformat is MapredParquetOutputFormat but this will surely be different for
HFile. right ?
What I meant was we need to keep a mapping (that identifies an
outputformat.) Yes, we do not use the output format but registering output
format as "MapredParquetOutputFormat" is misleading.
We can just keep an enum that identifies the output format for a each
storage type and use it here. Thats all is needed.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]