[HUDI-69] Support Spark Datasource for MOR table - RDD approach (#1848)
- This PR implements Spark Datasource for MOR table in the RDD approach. - Implemented SnapshotRelation - Implemented HudiMergeOnReadRDD - Implemented separate Iterator to handle merge and unmerge record reader. - Added TestMORDataSource to verify this feature. - Clean up test file name, add tests for mixed query type tests - We can now revert the change made in DefaultSource Co-authored-by: Vinoth Chandar <vchandar@confluent.io>
This commit is contained in:
@@ -23,7 +23,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.LogReaderUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.InputSplitUtils;
|
||||
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
@@ -148,12 +147,4 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
public Schema getHiveSchema() {
|
||||
return hiveSchema;
|
||||
}
|
||||
|
||||
public long getMaxCompactionMemoryInBytes() {
|
||||
// jobConf.getMemoryForMapTask() returns in MB
|
||||
return (long) Math
|
||||
.ceil(Double.parseDouble(jobConf.get(HoodieRealtimeConfig.COMPACTION_MEMORY_FRACTION_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_COMPACTION_MEMORY_FRACTION))
|
||||
* jobConf.getMemoryForMapTask() * 1024 * 1024L);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -69,7 +69,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
|
||||
split.getDeltaLogPaths(),
|
||||
usesCustomPayload ? getWriterSchema() : getReaderSchema(),
|
||||
split.getMaxCommitTime(),
|
||||
getMaxCompactionMemoryInBytes(),
|
||||
HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(jobConf),
|
||||
Boolean.parseBoolean(jobConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false,
|
||||
jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
|
||||
|
||||
@@ -63,7 +63,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
|
||||
* clients to consume.
|
||||
*
|
||||
* @param split File split
|
||||
* @param job Job Configuration
|
||||
* @param jobConf Job Configuration
|
||||
* @param realReader Parquet Reader
|
||||
*/
|
||||
public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job,
|
||||
@@ -72,14 +72,15 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
|
||||
this.parquetReader = new SafeParquetRecordReaderWrapper(realReader);
|
||||
// Iterator for consuming records from parquet file
|
||||
this.parquetRecordsIterator = new RecordReaderValueIterator<>(this.parquetReader);
|
||||
this.executor = new BoundedInMemoryExecutor<>(getMaxCompactionMemoryInBytes(), getParallelProducers(),
|
||||
this.executor = new BoundedInMemoryExecutor<>(
|
||||
HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(jobConf), getParallelProducers(),
|
||||
Option.empty(), x -> x, new DefaultSizeEstimator<>());
|
||||
// Consumer of this record reader
|
||||
this.iterator = this.executor.getQueue().iterator();
|
||||
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
|
||||
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), this.jobConf),
|
||||
split.getBasePath(), split.getDeltaLogPaths(), getReaderSchema(), split.getMaxCommitTime(),
|
||||
Boolean.parseBoolean(jobConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false, jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), record -> {
|
||||
Boolean.parseBoolean(this.jobConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false, this.jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), record -> {
|
||||
// convert Hoodie log record to Hadoop AvroWritable and buffer
|
||||
GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema()).get();
|
||||
ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema());
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.hadoop.utils;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
@@ -34,7 +35,6 @@ import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
|
||||
import org.apache.hudi.hadoop.realtime.RealtimeBootstrapBaseFileSplit;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileSplit;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
@@ -119,15 +119,15 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
|
||||
}
|
||||
|
||||
// Return parquet file with a list of log files in the same file group.
|
||||
public static Map<String, List<String>> groupLogsByBaseFile(Configuration conf, Stream<FileStatus> fileStatuses) {
|
||||
Map<Path, List<FileStatus>> partitionsToParquetSplits =
|
||||
fileStatuses.collect(Collectors.groupingBy(file -> file.getPath().getParent()));
|
||||
public static Map<HoodieBaseFile, List<String>> groupLogsByBaseFile(Configuration conf, List<HoodieBaseFile> fileStatuses) {
|
||||
Map<Path, List<HoodieBaseFile>> partitionsToParquetSplits =
|
||||
fileStatuses.stream().collect(Collectors.groupingBy(file -> file.getFileStatus().getPath().getParent()));
|
||||
// TODO(vc): Should we handle also non-hoodie splits here?
|
||||
Map<Path, HoodieTableMetaClient> partitionsToMetaClient = getTableMetaClientByBasePath(conf, partitionsToParquetSplits.keySet());
|
||||
|
||||
// for all unique split parents, obtain all delta files based on delta commit timeline,
|
||||
// grouped on file id
|
||||
Map<String, List<String>> resultMap = new HashMap<>();
|
||||
Map<HoodieBaseFile, List<String>> resultMap = new HashMap<>();
|
||||
partitionsToParquetSplits.keySet().forEach(partitionPath -> {
|
||||
// for each partition path obtain the data & log file groupings, then map back to inputsplits
|
||||
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
|
||||
@@ -144,15 +144,15 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
|
||||
.orElse(Stream.empty());
|
||||
|
||||
// subgroup splits again by file id & match with log files.
|
||||
Map<String, List<FileStatus>> groupedInputSplits = partitionsToParquetSplits.get(partitionPath).stream()
|
||||
.collect(Collectors.groupingBy(file -> FSUtils.getFileId(file.getPath().getName())));
|
||||
Map<String, List<HoodieBaseFile>> groupedInputSplits = partitionsToParquetSplits.get(partitionPath).stream()
|
||||
.collect(Collectors.groupingBy(file -> FSUtils.getFileId(file.getFileStatus().getPath().getName())));
|
||||
latestFileSlices.forEach(fileSlice -> {
|
||||
List<FileStatus> dataFileSplits = groupedInputSplits.get(fileSlice.getFileId());
|
||||
List<HoodieBaseFile> dataFileSplits = groupedInputSplits.get(fileSlice.getFileId());
|
||||
dataFileSplits.forEach(split -> {
|
||||
try {
|
||||
List<String> logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(logFile -> logFile.getPath().toString()).collect(Collectors.toList());
|
||||
resultMap.put(split.getPath().toString(), logFilePaths);
|
||||
resultMap.put(split, logFilePaths);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error creating hoodie real time split ", e);
|
||||
}
|
||||
|
||||
@@ -22,6 +22,7 @@ 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.hadoop.config.HoodieRealtimeConfig;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
@@ -43,6 +44,7 @@ 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.hadoop.mapred.JobConf;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
@@ -69,6 +71,17 @@ public class HoodieRealtimeRecordReaderUtils {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* get the max compaction memory in bytes from JobConf.
|
||||
*/
|
||||
public static long getMaxCompactionMemoryInBytes(JobConf jobConf) {
|
||||
// jobConf.getMemoryForMapTask() returns in MB
|
||||
return (long) Math
|
||||
.ceil(Double.parseDouble(jobConf.get(HoodieRealtimeConfig.COMPACTION_MEMORY_FRACTION_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_COMPACTION_MEMORY_FRACTION))
|
||||
* jobConf.getMemoryForMapTask() * 1024 * 1024L);
|
||||
}
|
||||
|
||||
/**
|
||||
* Prints a JSON representation of the ArrayWritable for easier debuggability.
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user