1
0

[HUDI-509] Renaming code in sync with cWiki restructuring (#1212)

- Storage Type replaced with Table Type (remaining instances)
 - View types replaced with query types;
 - ReadOptimized view referred as Snapshot Query
 - TableFileSystemView sub interfaces renamed to BaseFileOnly and Slice Views
 - HoodieDataFile renamed to HoodieBaseFile
 - Hive Sync tool will register RO tables for MOR with a `_ro` suffix
 - Datasource/Deltastreamer options renamed accordingly
 - Support fallback to old config values as well, so migration is painless
 - Config for controlling _ro suffix addition
 - Renaming DataFile to BaseFile across DTOs, HoodieFileSlice and AbstractTableFileSystemView
This commit is contained in:
vinoth chandar
2020-01-16 23:58:47 -08:00
committed by GitHub
parent 8a3a50309b
commit c2c0f6b13d
92 changed files with 907 additions and 822 deletions

View File

@@ -23,7 +23,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -293,7 +293,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst());
if (fileSliceOptional.isPresent()) {
FileSlice fs = fileSliceOptional.get();
Option<HoodieDataFile> df = fs.getDataFile();
Option<HoodieBaseFile> df = fs.getBaseFile();
if (operation.getDataFileName().isPresent()) {
String expPath = metaClient.getFs()
.getFileStatus(
@@ -448,7 +448,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
.orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1);
String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension())
.orElse(HoodieLogFile.DELTA_EXTENSION);
String parentPath = fileSliceForCompaction.getDataFile().map(df -> new Path(df.getPath()).getParent().toString())
String parentPath = fileSliceForCompaction.getBaseFile().map(df -> new Path(df.getPath()).getParent().toString())
.orElse(fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getPath().getParent().toString()).get());
for (HoodieLogFile toRepair : logFilesToRepair) {
int version = maxUsedVersion + 1;

View File

@@ -20,7 +20,7 @@ package org.apache.hudi;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
@@ -136,8 +136,8 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
private Option<String> convertToDataFilePath(Option<Pair<String, String>> partitionPathFileIDPair) {
if (partitionPathFileIDPair.isPresent()) {
HoodieDataFile dataFile = hoodieTable.getROFileSystemView()
.getLatestDataFile(partitionPathFileIDPair.get().getLeft(), partitionPathFileIDPair.get().getRight()).get();
HoodieBaseFile dataFile = hoodieTable.getBaseFileOnlyView()
.getLatestBaseFile(partitionPathFileIDPair.get().getLeft(), partitionPathFileIDPair.get().getRight()).get();
return Option.of(dataFile.getPath());
} else {
return Option.empty();

View File

@@ -26,7 +26,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
@@ -34,7 +34,7 @@ import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
@@ -592,9 +592,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
// Scan all partitions files with this commit time
LOG.info("Collecting latest files in partition path " + partitionPath);
ReadOptimizedView view = table.getROFileSystemView();
List<String> latestFiles = view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime)
.map(HoodieDataFile::getFileName).collect(Collectors.toList());
BaseFileOnlyView view = table.getBaseFileOnlyView();
List<String> latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, commitTime)
.map(HoodieBaseFile::getFileName).collect(Collectors.toList());
return new Tuple2<>(partitionPath, latestFiles);
}).collectAsMap();

View File

@@ -245,8 +245,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
hoodieTable.getMetaClient().getCommitsTimeline().filterCompletedInstants().lastInstant();
List<Pair<String, String>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) {
filteredFiles = hoodieTable.getROFileSystemView()
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
filteredFiles = hoodieTable.getBaseFileOnlyView()
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
.map(f -> Pair.of(partitionPath, f.getFileId())).collect(toList());
}
return filteredFiles.iterator();

View File

@@ -29,7 +29,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
import org.apache.hudi.common.table.TableFileSystemView.RealtimeView;
import org.apache.hudi.common.table.TableFileSystemView.SliceView;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
@@ -117,7 +117,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
if (doInit) {
this.partitionPath = record.getPartitionPath();
// extract some information from the first record
RealtimeView rtView = hoodieTable.getRTFileSystemView();
SliceView rtView = hoodieTable.getSliceView();
Option<FileSlice> fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
// Set the base commit time as the current commitTime for new inserts into log files
String baseInstantTime = instantTime;

View File

@@ -23,7 +23,7 @@ import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
@@ -76,7 +76,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> implements Seri
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
this.config = config;
this.fgIdToPendingCompactionOperations =
((SyncableFileSystemView) hoodieTable.getRTFileSystemView()).getPendingCompactionOperations()
((SyncableFileSystemView) hoodieTable.getSliceView()).getPendingCompactionOperations()
.map(entry -> Pair.of(
new HoodieFileGroupId(entry.getValue().getPartitionPath(), entry.getValue().getFileId()),
entry.getValue()))
@@ -148,7 +148,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> implements Seri
while (fileSliceIterator.hasNext() && keepVersions > 0) {
// Skip this most recent version
FileSlice nextSlice = fileSliceIterator.next();
Option<HoodieDataFile> dataFile = nextSlice.getDataFile();
Option<HoodieBaseFile> dataFile = nextSlice.getBaseFile();
if (dataFile.isPresent() && savepointedFiles.contains(dataFile.get().getFileName())) {
// do not clean up a savepoint data file
continue;
@@ -158,8 +158,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> implements Seri
// Delete the remaining files
while (fileSliceIterator.hasNext()) {
FileSlice nextSlice = fileSliceIterator.next();
if (nextSlice.getDataFile().isPresent()) {
HoodieDataFile dataFile = nextSlice.getDataFile().get();
if (nextSlice.getBaseFile().isPresent()) {
HoodieBaseFile dataFile = nextSlice.getBaseFile().get();
deletePaths.add(dataFile.getFileName());
}
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
@@ -212,7 +212,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> implements Seri
// Ensure there are more than 1 version of the file (we only clean old files from updates)
// i.e always spare the last commit.
for (FileSlice aSlice : fileSliceList) {
Option<HoodieDataFile> aFile = aSlice.getDataFile();
Option<HoodieBaseFile> aFile = aSlice.getBaseFile();
String fileCommitTime = aSlice.getBaseInstantTime();
if (aFile.isPresent() && savepointedFiles.contains(aFile.get().getFileName())) {
// do not clean up a savepoint data file

View File

@@ -19,7 +19,7 @@
package org.apache.hudi.io;
import org.apache.hudi.common.bloom.filter.BloomFilter;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.util.HoodieTimer;
@@ -113,7 +113,7 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends Hoodie
LOG.debug("#The candidate row keys for " + partitionPathFilePair + " => " + candidateRecordKeys);
}
HoodieDataFile dataFile = getLatestDataFile();
HoodieBaseFile dataFile = getLatestDataFile();
List<String> matchingKeys =
checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys, new Path(dataFile.getPath()));
LOG.info(

View File

@@ -19,7 +19,7 @@
package org.apache.hudi.io;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
@@ -73,14 +73,14 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
Iterator<HoodieRecord<T>> recordItr, String fileId) {
super(config, commitTime, fileId, hoodieTable);
String partitionPath = init(fileId, recordItr);
init(fileId, partitionPath, hoodieTable.getROFileSystemView().getLatestDataFile(partitionPath, fileId).get());
init(fileId, partitionPath, hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get());
}
/**
* Called by compactor code path.
*/
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
Map<String, HoodieRecord<T>> keyToNewRecords, String fileId, HoodieDataFile dataFileToBeMerged) {
Map<String, HoodieRecord<T>> keyToNewRecords, String fileId, HoodieBaseFile dataFileToBeMerged) {
super(config, commitTime, fileId, hoodieTable);
this.keyToNewRecords = keyToNewRecords;
this.useWriterSchema = true;
@@ -154,7 +154,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
/**
* Extract old file path, initialize StorageWriter and WriteStatus.
*/
private void init(String fileId, String partitionPath, HoodieDataFile dataFileToBeMerged) {
private void init(String fileId, String partitionPath, HoodieBaseFile dataFileToBeMerged) {
LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" + fileId);
this.writtenRecordKeys = new HashSet<>();
writeStatus.setStat(new HoodieWriteStat());

View File

@@ -18,7 +18,7 @@
package org.apache.hudi.io;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.ParquetUtils;
import org.apache.hudi.common.util.collection.Pair;
@@ -38,7 +38,7 @@ public class HoodieRangeInfoHandle<T extends HoodieRecordPayload> extends Hoodie
}
public String[] getMinMaxKeys() {
HoodieDataFile dataFile = getLatestDataFile();
HoodieBaseFile dataFile = getLatestDataFile();
return ParquetUtils.readMinMaxRecordKeys(hoodieTable.getHadoopConf(), new Path(dataFile.getPath()));
}
}

View File

@@ -18,7 +18,7 @@
package org.apache.hudi.io;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
@@ -52,8 +52,8 @@ public abstract class HoodieReadHandle<T extends HoodieRecordPayload> extends Ho
return partitionPathFilePair.getRight();
}
protected HoodieDataFile getLatestDataFile() {
return hoodieTable.getROFileSystemView()
.getLatestDataFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get();
protected HoodieBaseFile getLatestDataFile() {
return hoodieTable.getBaseFileOnlyView()
.getLatestBaseFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get();
}
}

View File

@@ -22,14 +22,14 @@ import org.apache.hudi.WriteStatus;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView.RealtimeView;
import org.apache.hudi.common.table.TableFileSystemView.SliceView;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.FSUtils;
@@ -66,15 +66,15 @@ import java.util.stream.StreamSupport;
import static java.util.stream.Collectors.toList;
/**
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all possible compactions,
* Compacts a hoodie table with merge on read storage. Computes all possible compactions,
* passes it through a CompactionFilter and executes all the compactions and writes a new version of base files and make
* a normal commit
*
* @see HoodieCompactor
*/
public class HoodieRealtimeTableCompactor implements HoodieCompactor {
public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
private static final Logger LOG = LogManager.getLogger(HoodieRealtimeTableCompactor.class);
private static final Logger LOG = LogManager.getLogger(HoodieMergeOnReadTableCompactor.class);
// Accumulator to keep track of total log files for a table
private AccumulatorV2<Long, Long> totalLogFiles;
// Accumulator to keep track of total log file slices for a table
@@ -128,7 +128,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return Lists.<WriteStatus>newArrayList();
}
Option<HoodieDataFile> oldDataFileOpt =
Option<HoodieBaseFile> oldDataFileOpt =
operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath());
// Compacting is very similar to applying updates to existing file
@@ -170,7 +170,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
jsc.sc().register(totalFileSlices);
Preconditions.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
"HoodieRealtimeTableCompactor can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not "
"Can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not "
+ hoodieTable.getMetaClient().getTableType().name());
// TODO : check if maxMemory is not greater than JVM or spark.executor memory
@@ -188,7 +188,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return null;
}
RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
SliceView fileSystemView = hoodieTable.getSliceView();
LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
List<HoodieCompactionOperation> operations = jsc.parallelize(partitionPaths, partitionPaths.size())
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
@@ -201,7 +201,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
// for spark Map operations and collecting them finally in Avro generated classes for storing
// into meta files.
Option<HoodieDataFile> dataFile = s.getDataFile();
Option<HoodieBaseFile> dataFile = s.getBaseFile();
return new CompactionOperation(dataFile, partitionPath, logFiles,
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
}).filter(c -> !c.getDeltaFileNames().isEmpty()).collect(toList()).iterator())

View File

@@ -20,13 +20,13 @@ package org.apache.hudi.io.compact.strategy;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
import org.apache.hudi.io.compact.HoodieMergeOnReadTableCompactor;
import com.google.common.collect.Maps;
@@ -40,7 +40,7 @@ import java.util.Map;
* compaction operation to run in a single compaction. Implementation of CompactionStrategy cannot hold any state.
* Difference instantiations can be passed in every time
*
* @see HoodieRealtimeTableCompactor
* @see HoodieMergeOnReadTableCompactor
*/
public abstract class CompactionStrategy implements Serializable {
@@ -59,7 +59,7 @@ public abstract class CompactionStrategy implements Serializable {
* @param logFiles - List of log files to compact with the base file
* @return Map[String, Object] - metrics captured
*/
public Map<String, Double> captureMetrics(HoodieWriteConfig writeConfig, Option<HoodieDataFile> dataFile,
public Map<String, Double> captureMetrics(HoodieWriteConfig writeConfig, Option<HoodieBaseFile> dataFile,
String partitionPath, List<HoodieLogFile> logFiles) {
Map<String, Double> metrics = Maps.newHashMap();
Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();

View File

@@ -20,7 +20,7 @@ package org.apache.hudi.io.compact.strategy;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
@@ -43,7 +43,7 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat
private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE";
@Override
public Map<String, Double> captureMetrics(HoodieWriteConfig config, Option<HoodieDataFile> dataFile,
public Map<String, Double> captureMetrics(HoodieWriteConfig config, Option<HoodieBaseFile> dataFile,
String partitionPath, List<HoodieLogFile> logFiles) {
Map<String, Double> metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles);

View File

@@ -26,7 +26,7 @@ import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
@@ -84,7 +84,9 @@ import java.util.stream.Collectors;
import scala.Tuple2;
/**
* Implementation of a very heavily read-optimized Hoodie Table where.
* Implementation of a very heavily read-optimized Hoodie Table where, all data is stored in base files, with
* zero read amplification.
*
* <p>
* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing file, to expand it
* <p>
@@ -181,7 +183,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieDataFile oldDataFile) throws IOException {
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
// these are updates
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, keyToNewRecords, oldDataFile);
return handleUpdateInternal(upsertHandle, commitTime, fileId);
@@ -223,7 +225,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieDataFile dataFileToBeMerged) {
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileId, dataFileToBeMerged);
}
@@ -685,10 +687,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
if (!commitTimeline.empty()) { // if we have some commits
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
List<HoodieDataFile> allFiles = getROFileSystemView()
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList());
List<HoodieBaseFile> allFiles = getBaseFileOnlyView()
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList());
for (HoodieDataFile file : allFiles) {
for (HoodieBaseFile file : allFiles) {
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
String filename = file.getFileName();
SmallFile sf = new SmallFile();

View File

@@ -40,7 +40,7 @@ import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.func.MergeOnReadLazyInsertIterable;
import org.apache.hudi.io.HoodieAppendHandle;
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
import org.apache.hudi.io.compact.HoodieMergeOnReadTableCompactor;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@@ -61,7 +61,8 @@ import java.util.Objects;
import java.util.stream.Collectors;
/**
* Implementation of a more real-time read-optimized Hoodie Table where
* Implementation of a more real-time Hoodie Table the provides tradeoffs on read and write cost/amplification.
*
* <p>
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
* smallest existing file, to expand it
@@ -142,10 +143,10 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
}
LOG.info("Compacting merge on read table " + config.getBasePath());
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor();
try {
return compactor.generateCompactionPlan(jsc, this, config, instantTime,
((SyncableFileSystemView) getRTFileSystemView()).getPendingCompactionOperations()
((SyncableFileSystemView) getSliceView()).getPendingCompactionOperations()
.map(instantTimeCompactionopPair -> instantTimeCompactionopPair.getValue().getFileGroupId())
.collect(Collectors.toSet()));
@@ -157,7 +158,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
@Override
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionInstantTime,
HoodieCompactionPlan compactionPlan) {
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor();
try {
return compactor.compact(jsc, compactionPlan, this, config, compactionInstantTime);
} catch (IOException e) {
@@ -344,10 +345,10 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
if (!index.canIndexLogFiles()) {
// TODO : choose last N small files since there can be multiple small files written to a single partition
// by different spark partitions in a single batch
Option<FileSlice> smallFileSlice = Option.fromJavaOptional(getRTFileSystemView()
Option<FileSlice> smallFileSlice = Option.fromJavaOptional(getSliceView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false)
.filter(fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getDataFile().get().getFileSize() < config.getParquetSmallFileLimit())
.min((FileSlice left, FileSlice right) -> left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() ? -1 : 1));
.filter(fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getBaseFile().get().getFileSize() < config.getParquetSmallFileLimit())
.min((FileSlice left, FileSlice right) -> left.getBaseFile().get().getFileSize() < right.getBaseFile().get().getFileSize() ? -1 : 1));
if (smallFileSlice.isPresent()) {
allSmallFileSlices.add(smallFileSlice.get());
}
@@ -355,7 +356,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
// If we can index log files, we can add more inserts to log files for fileIds including those under
// pending compaction.
List<FileSlice> allFileSlices =
getRTFileSystemView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true)
getSliceView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true)
.collect(Collectors.toList());
for (FileSlice fileSlice : allFileSlices) {
if (isSmallFile(fileSlice)) {
@@ -366,9 +367,9 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
// Create SmallFiles from the eligible file slices
for (FileSlice smallFileSlice : allSmallFileSlices) {
SmallFile sf = new SmallFile();
if (smallFileSlice.getDataFile().isPresent()) {
if (smallFileSlice.getBaseFile().isPresent()) {
// TODO : Move logic of file name, file id, base commit time handling inside file slice
String filename = smallFileSlice.getDataFile().get().getFileName();
String filename = smallFileSlice.getBaseFile().get().getFileName();
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
sf.sizeBytes = getTotalFileSize(smallFileSlice);
smallFileLocations.add(sf);
@@ -394,10 +395,10 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
}
private long getTotalFileSize(FileSlice fileSlice) {
if (!fileSlice.getDataFile().isPresent()) {
if (!fileSlice.getBaseFile().isPresent()) {
return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
} else {
return fileSlice.getDataFile().get().getFileSize()
return fileSlice.getBaseFile().get().getFileSize()
+ convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
}
}
@@ -428,7 +429,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
// used to write the new log files. In this case, the commit time for the log file is the compaction requested time.
// But the index (global) might store the baseCommit of the parquet and not the requested, hence get the
// baseCommit always by listing the file slice
Map<String, String> fileIdToBaseCommitTimeForLogMap = this.getRTFileSystemView().getLatestFileSlices(partitionPath)
Map<String, String> fileIdToBaseCommitTimeForLogMap = this.getSliceView().getLatestFileSlices(partitionPath)
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime));
return commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> {

View File

@@ -33,6 +33,8 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
import org.apache.hudi.common.table.TableFileSystemView;
import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.TableFileSystemView.SliceView;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.FileSystemViewManager;
@@ -145,16 +147,16 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
}
/**
* Get the read optimized view of the file system for this table.
* Get the base file only view of the file system for this table.
*/
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
public BaseFileOnlyView getBaseFileOnlyView() {
return getViewManager().getFileSystemView(metaClient.getBasePath());
}
/**
* Get the real time view of the file system for this table.
* Get the full view of the file system for this table.
*/
public TableFileSystemView.RealtimeView getRTFileSystemView() {
public SliceView getSliceView() {
return getViewManager().getFileSystemView(metaClient.getBasePath());
}