[HUDI-684] Introduced abstraction for writing and reading different types of base file formats. (#1687)
Notable changes:
1. HoodieFileWriter and HoodieFileReader abstractions for writer/reader side of a base file format
2. HoodieDataBlock abstraction for creation specific data blocks for base file formats. (e.g. Parquet has HoodieAvroDataBlock)
3. All hardocded references to Parquet / Parquet based classes have been abstracted to call methods which accept a base file format
4. HiveSyncTool accepts the base file format as a CLI parameter
5. HoodieDeltaStreamer accepts the base file format as a CLI parameter
6. HoodieSparkSqlWriter accepts the base file format as a parameter
This commit is contained in:
@@ -35,6 +35,9 @@ public class HiveSyncConfig implements Serializable {
|
||||
@Parameter(names = {"--table"}, description = "name of the target table in Hive", required = true)
|
||||
public String tableName;
|
||||
|
||||
@Parameter(names = {"--base-file-format"}, description = "Format of the base files (PARQUET (or) HFILE)")
|
||||
public String baseFileFormat = "PARQUET";
|
||||
|
||||
@Parameter(names = {"--user"}, description = "Hive username", required = true)
|
||||
public String hiveUser;
|
||||
|
||||
|
||||
@@ -19,10 +19,10 @@
|
||||
package org.apache.hudi.hive;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.InvalidTableException;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
|
||||
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
|
||||
import org.apache.hudi.hive.util.HiveSchemaUtil;
|
||||
@@ -32,8 +32,6 @@ 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.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
@@ -146,21 +144,24 @@ public class HiveSyncTool {
|
||||
// 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();
|
||||
}
|
||||
|
||||
String outputFormatClassName = HoodieInputFormatUtils.getOutputFormatClassName(baseFileFormat);
|
||||
String serDeFormatClassName = HoodieInputFormatUtils.getSerDeClassName(baseFileFormat);
|
||||
|
||||
// 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, outputFormatClassName, serDeFormatClassName);
|
||||
} else {
|
||||
// Check if the table schema has evolved
|
||||
Map<String, String> tableSchema = hoodieHiveClient.getTableSchema(tableName);
|
||||
|
||||
Reference in New Issue
Block a user