1
0

[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:
Prashant Wason
2020-06-25 23:46:55 -07:00
committed by GitHub
parent 5e47673341
commit 2603cfb33e
55 changed files with 1086 additions and 466 deletions

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.hadoop;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -101,7 +102,8 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
setInputPaths(job, snapshotPaths.toArray(new Path[snapshotPaths.size()]));
FileStatus[] fileStatuses = super.listStatus(job);
Map<HoodieTableMetaClient, List<FileStatus>> groupedFileStatus =
HoodieInputFormatUtils.groupFileStatusForSnapshotPaths(fileStatuses, tableMetaClientMap.values());
HoodieInputFormatUtils.groupFileStatusForSnapshotPaths(fileStatuses,
HoodieFileFormat.PARQUET.getFileExtension(), tableMetaClientMap.values());
LOG.info("Found a total of " + groupedFileStatus.size() + " groups");
for (Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry : groupedFileStatus.entrySet()) {
List<FileStatus> result = HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, entry.getKey(), entry.getValue());

View File

@@ -24,7 +24,7 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieCombineRealtimeRecordReader;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -952,7 +952,11 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
ValidationUtils.checkArgument(split instanceof HoodieCombineRealtimeFileSplit, "Only "
+ HoodieCombineRealtimeFileSplit.class.getName() + " allowed, found " + split.getClass().getName());
for (InputSplit inputSplit : ((HoodieCombineRealtimeFileSplit) split).getRealtimeFileSplits()) {
recordReaders.add(new HoodieParquetRealtimeInputFormat().getRecordReader(inputSplit, job, reporter));
if (split.getPaths().length == 0) {
continue;
}
FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(split.getPath(0).toString(), true, job);
recordReaders.add(inputFormat.getRecordReader(inputSplit, job, reporter));
}
return new HoodieCombineRealtimeRecordReader(job, split, recordReaders);
}

View File

@@ -32,8 +32,6 @@ import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.mapred.JobConf;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.schema.MessageType;
import java.io.IOException;
import java.util.ArrayList;
@@ -50,7 +48,6 @@ public abstract class AbstractRealtimeRecordReader {
protected final HoodieRealtimeFileSplit split;
protected final JobConf jobConf;
private final MessageType baseFileSchema;
protected final boolean usesCustomPayload;
// Schema handles
private Schema readerSchema;
@@ -66,7 +63,6 @@ public abstract class AbstractRealtimeRecordReader {
try {
this.usesCustomPayload = usesCustomPayload();
LOG.info("usesCustomPayload ==> " + this.usesCustomPayload);
baseFileSchema = HoodieRealtimeRecordReaderUtils.readSchema(jobConf, split.getPath());
init();
} catch (IOException e) {
throw new HoodieIOException("Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
@@ -88,7 +84,7 @@ public abstract class AbstractRealtimeRecordReader {
Schema schemaFromLogFile =
LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaLogPaths(), jobConf);
if (schemaFromLogFile == null) {
writerSchema = new AvroSchemaConverter().convert(baseFileSchema);
writerSchema = HoodieRealtimeRecordReaderUtils.readSchema(jobConf, split.getPath());
LOG.debug("Writer Schema From Parquet => " + writerSchema.getFields());
} else {
writerSchema = schemaFromLogFile;

View File

@@ -18,8 +18,10 @@
package org.apache.hudi.hadoop.utils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
@@ -30,11 +32,15 @@ import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
import org.apache.log4j.LogManager;
@@ -61,6 +67,54 @@ public class HoodieInputFormatUtils {
private static final Logger LOG = LogManager.getLogger(HoodieInputFormatUtils.class);
public static FileInputFormat getInputFormat(HoodieFileFormat baseFileFormat, boolean realtime, Configuration conf) {
switch (baseFileFormat) {
case PARQUET:
if (realtime) {
HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat();
inputFormat.setConf(conf);
return inputFormat;
} else {
HoodieParquetInputFormat inputFormat = new HoodieParquetInputFormat();
inputFormat.setConf(conf);
return inputFormat;
}
default:
throw new HoodieIOException("Hoodie InputFormat not implemented for base file format " + baseFileFormat);
}
}
public static String getInputFormatClassName(HoodieFileFormat baseFileFormat, boolean realtime, Configuration conf) {
FileInputFormat inputFormat = getInputFormat(baseFileFormat, realtime, conf);
return inputFormat.getClass().getName();
}
public static String getOutputFormatClassName(HoodieFileFormat baseFileFormat) {
switch (baseFileFormat) {
case PARQUET:
return MapredParquetOutputFormat.class.getName();
default:
throw new HoodieIOException("No OutputFormat for base file format " + baseFileFormat);
}
}
public static String getSerDeClassName(HoodieFileFormat baseFileFormat) {
switch (baseFileFormat) {
case PARQUET:
return ParquetHiveSerDe.class.getName();
default:
throw new HoodieIOException("No SerDe for base file format " + baseFileFormat);
}
}
public static FileInputFormat getInputFormat(String path, boolean realtime, Configuration conf) {
final String extension = FSUtils.getFileExtension(path.toString());
if (extension.equals(HoodieFileFormat.PARQUET.getFileExtension())) {
return getInputFormat(HoodieFileFormat.PARQUET, realtime, conf);
}
throw new HoodieIOException("Hoodie InputFormat not implemented for base file of type " + extension);
}
/**
* Filter any specific instants that we do not want to process.
* example timeline:
@@ -255,19 +309,20 @@ public class HoodieInputFormatUtils {
* Takes in a list of filesStatus and a list of table metadatas. Groups the files status list
* based on given table metadata.
* @param fileStatuses
* @param fileExtension
* @param metaClientList
* @return
* @throws IOException
*/
public static Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatusForSnapshotPaths(
FileStatus[] fileStatuses, Collection<HoodieTableMetaClient> metaClientList) {
FileStatus[] fileStatuses, String fileExtension, Collection<HoodieTableMetaClient> metaClientList) {
// This assumes the paths for different tables are grouped together
Map<HoodieTableMetaClient, List<FileStatus>> grouped = new HashMap<>();
HoodieTableMetaClient metadata = null;
for (FileStatus status : fileStatuses) {
Path inputPath = status.getPath();
if (!inputPath.getName().endsWith(".parquet")) {
//FIXME(vc): skip non parquet files for now. This wont be needed once log file name start
if (!inputPath.getName().endsWith(fileExtension)) {
//FIXME(vc): skip non data files for now. This wont be needed once log file name start
// with "."
continue;
}

View File

@@ -22,7 +22,8 @@ import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericArray;
@@ -40,8 +41,6 @@ import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.schema.MessageType;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -57,14 +56,14 @@ import java.util.stream.Collectors;
public class HoodieRealtimeRecordReaderUtils {
/**
* Reads the schema from the parquet file. This is different from ParquetUtils as it uses the twitter parquet to
* support hive 1.1.0
* Reads the schema from the base file.
*/
public static MessageType readSchema(Configuration conf, Path parquetFilePath) {
public static Schema readSchema(Configuration conf, Path filePath) {
try {
return ParquetFileReader.readFooter(conf, parquetFilePath).getFileMetaData().getSchema();
HoodieFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf, filePath);
return storageReader.getSchema();
} catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e);
throw new HoodieIOException("Failed to read schema from " + filePath, e);
}
}