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

@@ -60,7 +60,7 @@ public class CompactionOperation implements Serializable {
this.metrics = metrics;
}
public CompactionOperation(Option<HoodieDataFile> dataFile, String partitionPath, List<HoodieLogFile> logFiles,
public CompactionOperation(Option<HoodieBaseFile> dataFile, String partitionPath, List<HoodieLogFile> logFiles,
Map<String, Double> metrics) {
if (dataFile.isPresent()) {
this.baseInstantTime = dataFile.get().getCommitTime();
@@ -111,9 +111,9 @@ public class CompactionOperation implements Serializable {
return id;
}
public Option<HoodieDataFile> getBaseFile(String basePath, String partitionPath) {
public Option<HoodieBaseFile> getBaseFile(String basePath, String partitionPath) {
Path dirPath = FSUtils.getPartitionPath(basePath, partitionPath);
return dataFileName.map(df -> new HoodieDataFile(new Path(dirPath, df).toString()));
return dataFileName.map(df -> new HoodieBaseFile(new Path(dirPath, df).toString()));
}
/**

View File

@@ -44,7 +44,7 @@ public class FileSlice implements Serializable {
/**
* data file, with the compacted data, for this slice.
*/
private HoodieDataFile dataFile;
private HoodieBaseFile baseFile;
/**
* List of appendable log files with real time data - Sorted with greater log version first - Always empty for
@@ -59,12 +59,12 @@ public class FileSlice implements Serializable {
public FileSlice(HoodieFileGroupId fileGroupId, String baseInstantTime) {
this.fileGroupId = fileGroupId;
this.baseInstantTime = baseInstantTime;
this.dataFile = null;
this.baseFile = null;
this.logFiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator());
}
public void setDataFile(HoodieDataFile dataFile) {
this.dataFile = dataFile;
public void setBaseFile(HoodieBaseFile baseFile) {
this.baseFile = baseFile;
}
public void addLogFile(HoodieLogFile logFile) {
@@ -91,8 +91,8 @@ public class FileSlice implements Serializable {
return fileGroupId;
}
public Option<HoodieDataFile> getDataFile() {
return Option.ofNullable(dataFile);
public Option<HoodieBaseFile> getBaseFile() {
return Option.ofNullable(baseFile);
}
public Option<HoodieLogFile> getLatestLogFile() {
@@ -105,7 +105,7 @@ public class FileSlice implements Serializable {
* @return
*/
public boolean isEmpty() {
return (dataFile == null) && (logFiles.isEmpty());
return (baseFile == null) && (logFiles.isEmpty());
}
@Override
@@ -113,7 +113,7 @@ public class FileSlice implements Serializable {
final StringBuilder sb = new StringBuilder("FileSlice {");
sb.append("fileGroupId=").append(fileGroupId);
sb.append(", baseCommitTime=").append(baseInstantTime);
sb.append(", dataFile='").append(dataFile).append('\'');
sb.append(", baseFile='").append(baseFile).append('\'');
sb.append(", logFiles='").append(logFiles).append('\'');
sb.append('}');
return sb.toString();
@@ -129,7 +129,7 @@ public class FileSlice implements Serializable {
}
FileSlice slice = (FileSlice) o;
return Objects.equals(fileGroupId, slice.fileGroupId) && Objects.equals(baseInstantTime, slice.baseInstantTime)
&& Objects.equals(dataFile, slice.dataFile) && Objects.equals(logFiles, slice.logFiles);
&& Objects.equals(baseFile, slice.baseFile) && Objects.equals(logFiles, slice.logFiles);
}
@Override

View File

@@ -27,21 +27,21 @@ import java.io.Serializable;
import java.util.Objects;
/**
* Hoodie data file.
* Hoodie base file.
*/
public class HoodieDataFile implements Serializable {
public class HoodieBaseFile implements Serializable {
private transient FileStatus fileStatus;
private final String fullPath;
private long fileLen;
public HoodieDataFile(FileStatus fileStatus) {
public HoodieBaseFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
this.fullPath = fileStatus.getPath().toString();
this.fileLen = fileStatus.getLen();
}
public HoodieDataFile(String filePath) {
public HoodieBaseFile(String filePath) {
this.fileStatus = null;
this.fullPath = filePath;
this.fileLen = -1;
@@ -87,7 +87,7 @@ public class HoodieDataFile implements Serializable {
if (o == null || getClass() != o.getClass()) {
return false;
}
HoodieDataFile dataFile = (HoodieDataFile) o;
HoodieBaseFile dataFile = (HoodieBaseFile) o;
return Objects.equals(fullPath, dataFile.fullPath);
}

View File

@@ -82,11 +82,11 @@ public class HoodieFileGroup implements Serializable {
/**
* Add a new datafile into the file group.
*/
public void addDataFile(HoodieDataFile dataFile) {
public void addBaseFile(HoodieBaseFile dataFile) {
if (!fileSlices.containsKey(dataFile.getCommitTime())) {
fileSlices.put(dataFile.getCommitTime(), new FileSlice(fileGroupId, dataFile.getCommitTime()));
}
fileSlices.get(dataFile.getCommitTime()).setDataFile(dataFile);
fileSlices.get(dataFile.getCommitTime()).setBaseFile(dataFile);
}
/**
@@ -155,8 +155,8 @@ public class HoodieFileGroup implements Serializable {
/**
* Gets the latest data file.
*/
public Option<HoodieDataFile> getLatestDataFile() {
return Option.fromJavaOptional(getAllDataFiles().findFirst());
public Option<HoodieBaseFile> getLatestDataFile() {
return Option.fromJavaOptional(getAllBaseFiles().findFirst());
}
/**
@@ -187,8 +187,8 @@ public class HoodieFileGroup implements Serializable {
/**
* Stream of committed data files, sorted reverse commit time.
*/
public Stream<HoodieDataFile> getAllDataFiles() {
return getAllFileSlices().filter(slice -> slice.getDataFile().isPresent()).map(slice -> slice.getDataFile().get());
public Stream<HoodieBaseFile> getAllBaseFiles() {
return getAllFileSlices().filter(slice -> slice.getBaseFile().isPresent()).map(slice -> slice.getBaseFile().get());
}
@Override

View File

@@ -27,9 +27,6 @@ package org.apache.hudi.common.model;
* <p>
* MERGE_ON_READ - Speeds up upserts, by delaying merge until enough work piles up.
* <p>
* In the future, following might be added.
* <p>
* SIMPLE_LSM - A simple 2 level LSM tree.
*/
public enum HoodieTableType {
COPY_ON_WRITE, MERGE_ON_READ

View File

@@ -53,15 +53,19 @@ public class HoodieTableConfig implements Serializable {
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
@Deprecated
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
@Deprecated
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format";
public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format";
public static final String HOODIE_LOG_FILE_FORMAT_PROP_NAME = "hoodie.table.log.file.format";
public static final String HOODIE_TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder";
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET;
public static final HoodieFileFormat DEFAULT_LOG_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
public static final String DEFAULT_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
public static final Integer DEFAULT_TIMELINE_LAYOUT_VERSION = TimelineLayoutVersion.VERSION_0;
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
@@ -164,27 +168,33 @@ public class HoodieTableConfig implements Serializable {
}
/**
* Get the Read Optimized Storage Format.
* Get the base file storage format.
*
* @return HoodieFileFormat for the Read Optimized Storage format
* @return HoodieFileFormat for the base file Storage format
*/
public HoodieFileFormat getROFileFormat() {
public HoodieFileFormat getBaseFileFormat() {
if (props.containsKey(HOODIE_BASE_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME));
}
if (props.containsKey(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME));
}
return DEFAULT_RO_FILE_FORMAT;
return DEFAULT_BASE_FILE_FORMAT;
}
/**
* Get the Read Optimized Storage Format.
* Get the log Storage Format.
*
* @return HoodieFileFormat for the Read Optimized Storage format
* @return HoodieFileFormat for the log Storage format
*/
public HoodieFileFormat getRTFileFormat() {
public HoodieFileFormat getLogFileFormat() {
if (props.containsKey(HOODIE_LOG_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_LOG_FILE_FORMAT_PROP_NAME));
}
if (props.containsKey(HOODIE_RT_FILE_FORMAT_PROP_NAME)) {
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RT_FILE_FORMAT_PROP_NAME));
}
return DEFAULT_RT_FILE_FORMAT;
return DEFAULT_LOG_FILE_FORMAT;
}
/**

View File

@@ -299,7 +299,7 @@ public class HoodieTableMetaClient implements Serializable {
}
/**
* Helper method to initialize a given path, as a given storage type and table name.
* Helper method to initialize a given path, as a given type and table name.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
@@ -437,7 +437,7 @@ public class HoodieTableMetaClient implements Serializable {
case MERGE_ON_READ:
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
default:
throw new HoodieException("Could not commit on unknown storage type " + this.getTableType());
throw new HoodieException("Could not commit on unknown table type " + this.getTableType());
}
}

View File

@@ -18,12 +18,15 @@
package org.apache.hudi.common.table;
import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.TableFileSystemView.SliceView;
/**
* A consolidated file-system view interface exposing both realtime and read-optimized views along with
* A consolidated file-system view interface exposing both complete slice and basefile only views along with
* update operations.
*/
public interface SyncableFileSystemView
extends TableFileSystemView, TableFileSystemView.ReadOptimizedView, TableFileSystemView.RealtimeView {
extends TableFileSystemView, BaseFileOnlyView, SliceView {

View File

@@ -20,7 +20,7 @@ package org.apache.hudi.common.table;
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.HoodieFileGroup;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
@@ -37,57 +37,57 @@ import java.util.stream.Stream;
public interface TableFileSystemView {
/**
* ReadOptimizedView with methods to only access latest version of file for the instant(s) passed.
* Methods to only access latest version of file for the instant(s) passed.
*/
interface ReadOptimizedViewWithLatestSlice {
interface BaseFileOnlyViewWithLatestSlice {
/**
* Stream all the latest data files in the given partition.
*/
Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
Stream<HoodieBaseFile> getLatestBaseFiles(String partitionPath);
/**
* Get Latest data file for a partition and file-Id.
*/
Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId);
Option<HoodieBaseFile> getLatestBaseFile(String partitionPath, String fileId);
/**
* Stream all the latest data files, in the file system view.
*/
Stream<HoodieDataFile> getLatestDataFiles();
Stream<HoodieBaseFile> getLatestBaseFiles();
/**
* Stream all the latest version data files in the given partition with precondition that commitTime(file) before
* maxCommitTime.
*/
Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime);
Stream<HoodieBaseFile> getLatestBaseFilesBeforeOrOn(String partitionPath, String maxCommitTime);
/**
* Stream all the latest data files pass.
*/
Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn);
Stream<HoodieBaseFile> getLatestBaseFilesInRange(List<String> commitsToReturn);
}
/**
* ReadOptimizedView - methods to provide a view of columnar data files only.
* Methods to provide a view of base files only.
*/
interface ReadOptimizedView extends ReadOptimizedViewWithLatestSlice {
interface BaseFileOnlyView extends BaseFileOnlyViewWithLatestSlice {
/**
* Stream all the data file versions grouped by FileId for a given partition.
*/
Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
Stream<HoodieBaseFile> getAllBaseFiles(String partitionPath);
/**
* Get the version of data file matching the instant time in the given partition.
*/
Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId);
Option<HoodieBaseFile> getBaseFileOn(String partitionPath, String instantTime, String fileId);
}
/**
* RealtimeView with methods to only access latest version of file-slice for the instant(s) passed.
* Methods to only access latest version of file-slice for the instant(s) passed.
*/
interface RealtimeViewWithLatestSlice {
interface SliceViewWithLatestSlice {
/**
* Stream all the latest file slices in the given partition.
@@ -131,9 +131,9 @@ public interface TableFileSystemView {
}
/**
* RealtimeView - methods to access a combination of columnar data files + log files with real time data.
* Methods to access a combination of base files + log file slices.
*/
interface RealtimeView extends RealtimeViewWithLatestSlice {
interface SliceView extends SliceViewWithLatestSlice {
/**
* Stream all the file slices for a given partition, latest or not.

View File

@@ -18,7 +18,7 @@
package org.apache.hudi.common.table.timeline.dto;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
@@ -27,7 +27,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
* The data transfer object of data file.
*/
@JsonIgnoreProperties(ignoreUnknown = true)
public class DataFileDTO {
public class BaseFileDTO {
@JsonProperty("fileStatus")
private FileStatusDTO fileStatus;
@@ -36,27 +36,27 @@ public class DataFileDTO {
@JsonProperty("fileLen")
private long fileLen;
public static HoodieDataFile toHoodieDataFile(DataFileDTO dto) {
public static HoodieBaseFile toHoodieBaseFile(BaseFileDTO dto) {
if (null == dto) {
return null;
}
HoodieDataFile dataFile = null;
HoodieBaseFile baseFile;
if (null != dto.fileStatus) {
dataFile = new HoodieDataFile(FileStatusDTO.toFileStatus(dto.fileStatus));
baseFile = new HoodieBaseFile(FileStatusDTO.toFileStatus(dto.fileStatus));
} else {
dataFile = new HoodieDataFile(dto.fullPath);
dataFile.setFileLen(dto.fileLen);
baseFile = new HoodieBaseFile(dto.fullPath);
baseFile.setFileLen(dto.fileLen);
}
return dataFile;
return baseFile;
}
public static DataFileDTO fromHoodieDataFile(HoodieDataFile dataFile) {
public static BaseFileDTO fromHoodieBaseFile(HoodieBaseFile dataFile) {
if (null == dataFile) {
return null;
}
DataFileDTO dto = new DataFileDTO();
BaseFileDTO dto = new BaseFileDTO();
dto.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus());
dto.fullPath = dataFile.getPath();
dto.fileLen = dataFile.getFileLen();

View File

@@ -32,8 +32,8 @@ import java.util.stream.Collectors;
@JsonIgnoreProperties(ignoreUnknown = true)
public class FileSliceDTO {
@JsonProperty("dataFile")
DataFileDTO dataFile;
@JsonProperty("baseFile")
BaseFileDTO baseFile;
@JsonProperty("logFiles")
List<LogFileDTO> logFiles;
@JsonProperty("partition")
@@ -48,14 +48,14 @@ public class FileSliceDTO {
dto.partitionPath = slice.getPartitionPath();
dto.baseInstantTime = slice.getBaseInstantTime();
dto.fileId = slice.getFileId();
dto.dataFile = slice.getDataFile().map(DataFileDTO::fromHoodieDataFile).orElse(null);
dto.baseFile = slice.getBaseFile().map(BaseFileDTO::fromHoodieBaseFile).orElse(null);
dto.logFiles = slice.getLogFiles().map(LogFileDTO::fromHoodieLogFile).collect(Collectors.toList());
return dto;
}
public static FileSlice toFileSlice(FileSliceDTO dto) {
FileSlice slice = new FileSlice(dto.partitionPath, dto.baseInstantTime, dto.fileId);
slice.setDataFile(DataFileDTO.toHoodieDataFile(dto.dataFile));
slice.setBaseFile(BaseFileDTO.toHoodieBaseFile(dto.baseFile));
dto.logFiles.stream().forEach(lf -> slice.addLogFile(LogFileDTO.toHoodieLogFile(lf)));
return slice;
}

View File

@@ -20,7 +20,7 @@ package org.apache.hudi.common.table.view;
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.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
@@ -133,16 +133,16 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
*/
protected List<HoodieFileGroup> buildFileGroups(FileStatus[] statuses, HoodieTimeline timeline,
boolean addPendingCompactionFileSlice) {
return buildFileGroups(convertFileStatusesToDataFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline,
return buildFileGroups(convertFileStatusesToBaseFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline,
addPendingCompactionFileSlice);
}
protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieDataFile> dataFileStream,
protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieBaseFile> baseFileStream,
Stream<HoodieLogFile> logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) {
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles =
dataFileStream.collect(Collectors.groupingBy((dataFile) -> {
String partitionPathStr = getPartitionPathFromFilePath(dataFile.getPath());
return Pair.of(partitionPathStr, dataFile.getFileId());
Map<Pair<String, String>, List<HoodieBaseFile>> baseFiles =
baseFileStream.collect(Collectors.groupingBy((baseFile) -> {
String partitionPathStr = getPartitionPathFromFilePath(baseFile.getPath());
return Pair.of(partitionPathStr, baseFile.getFileId());
}));
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = logFileStream.collect(Collectors.groupingBy((logFile) -> {
@@ -151,15 +151,15 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
return Pair.of(partitionPathStr, logFile.getFileId());
}));
Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
Set<Pair<String, String>> fileIdSet = new HashSet<>(baseFiles.keySet());
fileIdSet.addAll(logFiles.keySet());
List<HoodieFileGroup> fileGroups = new ArrayList<>();
fileIdSet.forEach(pair -> {
String fileId = pair.getValue();
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, timeline);
if (dataFiles.containsKey(pair)) {
dataFiles.get(pair).forEach(group::addDataFile);
if (baseFiles.containsKey(pair)) {
baseFiles.get(pair).forEach(group::addBaseFile);
}
if (logFiles.containsKey(pair)) {
logFiles.get(pair).forEach(group::addLogFile);
@@ -233,7 +233,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
storePartitionView(partitionPathStr, new ArrayList<>());
}
} catch (IOException e) {
throw new HoodieIOException("Failed to list data files in partition " + partitionPathStr, e);
throw new HoodieIOException("Failed to list base files in partition " + partitionPathStr, e);
}
} else {
LOG.debug("View already built for Partition :" + partitionPathStr + ", FOUND is ");
@@ -245,14 +245,14 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
/**
* Helper to convert file-status to data-files.
* Helper to convert file-status to base-files.
*
* @param statuses List of File-Status
*/
private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) {
private Stream<HoodieBaseFile> convertFileStatusesToBaseFiles(FileStatus[] statuses) {
Predicate<FileStatus> roFilePredicate = fileStatus -> fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new);
.contains(metaClient.getTableConfig().getBaseFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieBaseFile::new);
}
/**
@@ -262,23 +262,23 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
*/
private Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) {
Predicate<FileStatus> rtFilePredicate = fileStatus -> fileStatus.getPath().getName()
.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
.contains(metaClient.getTableConfig().getLogFileFormat().getFileExtension());
return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new);
}
/**
* With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those
* data-files.
* With async compaction, it is possible to see partial/complete base-files due to inflight-compactions, Ignore those
* base-files.
*
* @param dataFile Data File
* @param baseFile base File
*/
protected boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) {
final String partitionPath = getPartitionPathFromFilePath(dataFile.getPath());
protected boolean isBaseFileDueToPendingCompaction(HoodieBaseFile baseFile) {
final String partitionPath = getPartitionPathFromFilePath(baseFile.getPath());
Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, dataFile.getFileId()));
getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, baseFile.getFileId()));
return (compactionWithInstantTime.isPresent()) && (null != compactionWithInstantTime.get().getKey())
&& dataFile.getCommitTime().equals(compactionWithInstantTime.get().getKey());
&& baseFile.getCommitTime().equals(compactionWithInstantTime.get().getKey());
}
/**
@@ -296,15 +296,15 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
/**
* With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those
* data-files.
* With async compaction, it is possible to see partial/complete base-files due to inflight-compactions, Ignore those
* base-files.
*
* @param fileSlice File Slice
*/
protected FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) {
protected FileSlice filterBaseFileAfterPendingCompaction(FileSlice fileSlice) {
if (isFileSliceAfterPendingCompaction(fileSlice)) {
LOG.info("File Slice (" + fileSlice + ") is in pending compaction");
// Data file is filtered out of the file-slice as the corresponding compaction
// Base file is filtered out of the file-slice as the corresponding compaction
// instant not completed yet.
FileSlice transformed =
new FileSlice(fileSlice.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId());
@@ -325,38 +325,38 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
@Override
public final Stream<HoodieDataFile> getLatestDataFiles(String partitionStr) {
public final Stream<HoodieBaseFile> getLatestBaseFiles(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestDataFiles(partitionPath);
return fetchLatestBaseFiles(partitionPath);
} finally {
readLock.unlock();
}
}
@Override
public final Stream<HoodieDataFile> getLatestDataFiles() {
public final Stream<HoodieBaseFile> getLatestBaseFiles() {
try {
readLock.lock();
return fetchLatestDataFiles();
return fetchLatestBaseFiles();
} finally {
readLock.unlock();
}
}
@Override
public final Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionStr, String maxCommitTime) {
public final Stream<HoodieBaseFile> getLatestBaseFilesBeforeOrOn(String partitionStr, String maxCommitTime) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllStoredFileGroups(partitionPath)
.map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllDataFiles()
.filter(dataFile -> HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), maxCommitTime,
.map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles()
.filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst()))
.filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst()))
.filter(Option::isPresent).map(Option::get);
} finally {
readLock.unlock();
@@ -364,43 +364,43 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
@Override
public final Option<HoodieDataFile> getDataFileOn(String partitionStr, String instantTime, String fileId) {
public final Option<HoodieBaseFile> getBaseFileOn(String partitionStr, String instantTime, String fileId) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllDataFiles()
return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles()
.filter(
dataFile -> HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), instantTime, HoodieTimeline.EQUAL))
.filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst().orElse(null));
baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), instantTime, HoodieTimeline.EQUAL))
.filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null));
} finally {
readLock.unlock();
}
}
/**
* Get Latest data file for a partition and file-Id.
* Get Latest base file for a partition and file-Id.
*/
@Override
public final Option<HoodieDataFile> getLatestDataFile(String partitionStr, String fileId) {
public final Option<HoodieBaseFile> getLatestBaseFile(String partitionStr, String fileId) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestDataFile(partitionPath, fileId);
return fetchLatestBaseFile(partitionPath, fileId);
} finally {
readLock.unlock();
}
}
@Override
public final Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
public final Stream<HoodieBaseFile> getLatestBaseFilesInRange(List<String> commitsToReturn) {
try {
readLock.lock();
return fetchAllStoredFileGroups().map(fileGroup -> {
return Option.fromJavaOptional(
fileGroup.getAllDataFiles().filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
&& !isDataFileDueToPendingCompaction(dataFile)).findFirst());
fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime())
&& !isBaseFileDueToPendingCompaction(baseFile)).findFirst());
}).filter(Option::isPresent).map(Option::get);
} finally {
readLock.unlock();
@@ -408,14 +408,14 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
@Override
public final Stream<HoodieDataFile> getAllDataFiles(String partitionStr) {
public final Stream<HoodieBaseFile> getAllBaseFiles(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllDataFiles(partitionPath)
return fetchAllBaseFiles(partitionPath)
.filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime()))
.filter(df -> !isDataFileDueToPendingCompaction(df));
.filter(df -> !isBaseFileDueToPendingCompaction(df));
} finally {
readLock.unlock();
}
@@ -427,7 +427,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestFileSlices(partitionPath).map(fs -> filterDataFileAfterPendingCompaction(fs));
return fetchLatestFileSlices(partitionPath).map(this::filterBaseFileAfterPendingCompaction);
} finally {
readLock.unlock();
}
@@ -443,7 +443,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
Option<FileSlice> fs = fetchLatestFileSlice(partitionPath, fileId);
return fs.map(f -> filterDataFileAfterPendingCompaction(f));
return fs.map(f -> filterBaseFileAfterPendingCompaction(f));
} finally {
readLock.unlock();
}
@@ -480,7 +480,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
ensurePartitionLoadedCorrectly(partitionPath);
Stream<FileSlice> fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime);
if (includeFileSlicesInPendingCompaction) {
return fileSliceStream.map(fs -> filterDataFileAfterPendingCompaction(fs));
return fileSliceStream.map(fs -> filterBaseFileAfterPendingCompaction(fs));
} else {
return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId()));
}
@@ -653,33 +653,33 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
/**
* Default implementation for fetching latest data-files for the partition-path.
* Default implementation for fetching latest base-files for the partition-path.
*/
Stream<HoodieDataFile> fetchLatestDataFiles(final String partitionPath) {
return fetchAllStoredFileGroups(partitionPath).map(this::getLatestDataFile).filter(Option::isPresent)
Stream<HoodieBaseFile> fetchLatestBaseFiles(final String partitionPath) {
return fetchAllStoredFileGroups(partitionPath).map(this::getLatestBaseFile).filter(Option::isPresent)
.map(Option::get);
}
protected Option<HoodieDataFile> getLatestDataFile(HoodieFileGroup fileGroup) {
protected Option<HoodieBaseFile> getLatestBaseFile(HoodieFileGroup fileGroup) {
return Option
.fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst());
.fromJavaOptional(fileGroup.getAllBaseFiles().filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst());
}
/**
* Default implementation for fetching latest data-files across all partitions.
* Default implementation for fetching latest base-files across all partitions.
*/
Stream<HoodieDataFile> fetchLatestDataFiles() {
return fetchAllStoredFileGroups().map(this::getLatestDataFile).filter(Option::isPresent).map(Option::get);
Stream<HoodieBaseFile> fetchLatestBaseFiles() {
return fetchAllStoredFileGroups().map(this::getLatestBaseFile).filter(Option::isPresent).map(Option::get);
}
/**
* Default implementation for fetching all data-files for a partition.
* Default implementation for fetching all base-files for a partition.
*
* @param partitionPath partition-path
*/
Stream<HoodieDataFile> fetchAllDataFiles(String partitionPath) {
return fetchAllStoredFileGroups(partitionPath).map(HoodieFileGroup::getAllDataFiles)
.flatMap(dataFileList -> dataFileList);
Stream<HoodieBaseFile> fetchAllBaseFiles(String partitionPath) {
return fetchAllStoredFileGroups(partitionPath).map(HoodieFileGroup::getAllBaseFiles)
.flatMap(baseFileList -> baseFileList);
}
/**
@@ -719,8 +719,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) {
FileSlice merged = new FileSlice(penultimateSlice.getPartitionPath(), penultimateSlice.getBaseInstantTime(),
penultimateSlice.getFileId());
if (penultimateSlice.getDataFile().isPresent()) {
merged.setDataFile(penultimateSlice.getDataFile().get());
if (penultimateSlice.getBaseFile().isPresent()) {
merged.setBaseFile(penultimateSlice.getBaseFile().get());
}
// Add Log files from penultimate and last slices
penultimateSlice.getLogFiles().forEach(merged::addLogFile);
@@ -752,15 +752,15 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
}
/**
* Default implementation for fetching latest data-file.
* Default implementation for fetching latest base-file.
*
* @param partitionPath Partition path
* @param fileId File Id
* @return Data File if present
* @return base File if present
*/
protected Option<HoodieDataFile> fetchLatestDataFile(String partitionPath, String fileId) {
return Option
.fromJavaOptional(fetchLatestDataFiles(partitionPath).filter(fs -> fs.getFileId().equals(fileId)).findFirst());
protected Option<HoodieBaseFile> fetchLatestBaseFile(String partitionPath, String fileId) {
return Option.fromJavaOptional(fetchLatestBaseFiles(partitionPath)
.filter(fs -> fs.getFileId().equals(fileId)).findFirst());
}
/**

View File

@@ -25,7 +25,7 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
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.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTimeline;
@@ -318,13 +318,13 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
* Note that while finding the new data/log files added/removed, the path stored in metadata will be missing the
* base-path,scheme and authority. Ensure the matching process takes care of this discrepancy.
*/
Map<String, HoodieDataFile> viewDataFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get)
Map<String, HoodieBaseFile> viewDataFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.map(FileSlice::getBaseFile).filter(Option::isPresent).map(Option::get)
.map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
// Note: Delta Log Files and Data FIles can be empty when adding/removing pending compactions
Map<String, HoodieDataFile> deltaDataFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get)
Map<String, HoodieBaseFile> deltaDataFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.map(FileSlice::getBaseFile).filter(Option::isPresent).map(Option::get)
.map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));

View File

@@ -20,7 +20,7 @@ package org.apache.hudi.common.table.view;
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.HoodieFileGroup;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
@@ -120,39 +120,39 @@ public class PriorityBasedFileSystemView implements SyncableFileSystemView, Seri
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles(String partitionPath) {
return execute(partitionPath, preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles);
public Stream<HoodieBaseFile> getLatestBaseFiles(String partitionPath) {
return execute(partitionPath, preferredView::getLatestBaseFiles, secondaryView::getLatestBaseFiles);
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles() {
return execute(preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles);
public Stream<HoodieBaseFile> getLatestBaseFiles() {
return execute(preferredView::getLatestBaseFiles, secondaryView::getLatestBaseFiles);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
return execute(partitionPath, maxCommitTime, preferredView::getLatestDataFilesBeforeOrOn,
secondaryView::getLatestDataFilesBeforeOrOn);
public Stream<HoodieBaseFile> getLatestBaseFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
return execute(partitionPath, maxCommitTime, preferredView::getLatestBaseFilesBeforeOrOn,
secondaryView::getLatestBaseFilesBeforeOrOn);
}
@Override
public Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId) {
return execute(partitionPath, fileId, preferredView::getLatestDataFile, secondaryView::getLatestDataFile);
public Option<HoodieBaseFile> getLatestBaseFile(String partitionPath, String fileId) {
return execute(partitionPath, fileId, preferredView::getLatestBaseFile, secondaryView::getLatestBaseFile);
}
@Override
public Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId) {
return execute(partitionPath, instantTime, fileId, preferredView::getDataFileOn, secondaryView::getDataFileOn);
public Option<HoodieBaseFile> getBaseFileOn(String partitionPath, String instantTime, String fileId) {
return execute(partitionPath, instantTime, fileId, preferredView::getBaseFileOn, secondaryView::getBaseFileOn);
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
return execute(commitsToReturn, preferredView::getLatestDataFilesInRange, secondaryView::getLatestDataFilesInRange);
public Stream<HoodieBaseFile> getLatestBaseFilesInRange(List<String> commitsToReturn) {
return execute(commitsToReturn, preferredView::getLatestBaseFilesInRange, secondaryView::getLatestBaseFilesInRange);
}
@Override
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
return execute(partitionPath, preferredView::getAllDataFiles, secondaryView::getAllDataFiles);
public Stream<HoodieBaseFile> getAllBaseFiles(String partitionPath) {
return execute(partitionPath, preferredView::getAllBaseFiles, secondaryView::getAllBaseFiles);
}
@Override

View File

@@ -20,14 +20,14 @@ package org.apache.hudi.common.table.view;
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.HoodieFileGroup;
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.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO;
import org.apache.hudi.common.table.timeline.dto.DataFileDTO;
import org.apache.hudi.common.table.timeline.dto.BaseFileDTO;
import org.apache.hudi.common.table.timeline.dto.FileGroupDTO;
import org.apache.hudi.common.table.timeline.dto.FileSliceDTO;
import org.apache.hudi.common.table.timeline.dto.InstantDTO;
@@ -205,74 +205,74 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles(String partitionPath) {
public Stream<HoodieBaseFile> getLatestBaseFiles(String partitionPath) {
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILES_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
List<BaseFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILES_URL, paramsMap,
new TypeReference<List<BaseFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieDataFile> getLatestDataFiles() {
public Stream<HoodieBaseFile> getLatestBaseFiles() {
Map<String, String> paramsMap = getParams();
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_ALL_DATA_FILES, paramsMap,
new TypeReference<List<DataFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
List<BaseFileDTO> dataFiles = executeRequest(LATEST_ALL_DATA_FILES, paramsMap,
new TypeReference<List<BaseFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
public Stream<HoodieBaseFile> getLatestBaseFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
List<BaseFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, paramsMap,
new TypeReference<List<BaseFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId) {
public Option<HoodieBaseFile> getBaseFileOn(String partitionPath, String instantTime, String fileId) {
Map<String, String> paramsMap = getParamsWithAdditionalParams(partitionPath,
new String[] {INSTANT_PARAM, FILEID_PARAM}, new String[] {instantTime, fileId});
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {}, RequestMethod.GET);
return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst());
List<BaseFileDTO> dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap,
new TypeReference<List<BaseFileDTO>>() {}, RequestMethod.GET);
return Option.fromJavaOptional(dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile).findFirst());
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
public Stream<HoodieBaseFile> getLatestBaseFilesInRange(List<String> commitsToReturn) {
Map<String, String> paramsMap =
getParams(INSTANTS_PARAM, StringUtils.join(commitsToReturn.toArray(new String[0]), ","));
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_RANGE_INSTANT_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
List<BaseFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_RANGE_INSTANT_URL, paramsMap,
new TypeReference<List<BaseFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
}
@Override
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
public Stream<HoodieBaseFile> getAllBaseFiles(String partitionPath) {
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
try {
List<DataFileDTO> dataFiles =
executeRequest(ALL_DATA_FILES, paramsMap, new TypeReference<List<DataFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
List<BaseFileDTO> dataFiles =
executeRequest(ALL_DATA_FILES, paramsMap, new TypeReference<List<BaseFileDTO>>() {}, RequestMethod.GET);
return dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}
@@ -439,12 +439,12 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
}
@Override
public Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId) {
public Option<HoodieBaseFile> getLatestBaseFile(String partitionPath, String fileId) {
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
try {
List<DataFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap,
new TypeReference<List<DataFileDTO>>() {}, RequestMethod.GET);
return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst());
List<BaseFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap,
new TypeReference<List<BaseFileDTO>>() {}, RequestMethod.GET);
return Option.fromJavaOptional(dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile).findFirst());
} catch (IOException e) {
throw new HoodieRemoteException(e);
}

View File

@@ -20,7 +20,7 @@ package org.apache.hudi.common.table.view;
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.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
@@ -174,7 +174,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
rocksDB.writeBatch(batch -> {
fg.getAllFileSlicesIncludingInflight().forEach(fs -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
fs.getDataFile().ifPresent(df -> {
fs.getBaseFile().ifPresent(df -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
df);
});
@@ -218,8 +218,8 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
switch (mode) {
case ADD: {
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
fs.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
oldSlice.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df));
fs.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df));
Map<String, HoodieLogFile> newLogFiles = new HashMap<>(logFiles);
deltaLogFiles.entrySet().stream().filter(e -> !logFiles.containsKey(e.getKey()))
.forEach(p -> newLogFiles.put(p.getKey(), p.getValue()));
@@ -230,15 +230,15 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
case REMOVE: {
LOG.info("Removing old File Slice =" + fs);
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
fs.getDataFile().orElseGet(() -> {
oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
fs.getBaseFile().orElseGet(() -> {
oldSlice.getBaseFile().ifPresent(df -> newFileSlice.setBaseFile(df));
return null;
});
deltaLogFiles.keySet().stream().forEach(p -> logFiles.remove(p));
// Add remaining log files back
logFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
if (newFileSlice.getDataFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) {
if (newFileSlice.getBaseFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) {
LOG.info("Adding back new file-slice after remove FS=" + newFileSlice);
return newFileSlice;
}
@@ -250,7 +250,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
}
}).filter(Objects::nonNull).forEach(fs -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
fs.getDataFile().ifPresent(df -> {
fs.getBaseFile().ifPresent(df -> {
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
df);
});
@@ -266,8 +266,8 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
}
@Override
Stream<HoodieDataFile> fetchAllDataFiles(String partitionPath) {
return rocksDB.<HoodieDataFile>prefixSearch(schemaHelper.getColFamilyForView(),
Stream<HoodieBaseFile> fetchAllBaseFiles(String partitionPath) {
return rocksDB.<HoodieBaseFile>prefixSearch(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForDataFileViewByPartition(partitionPath)).map(Pair::getValue);
}
@@ -298,11 +298,11 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
}
@Override
protected Option<HoodieDataFile> fetchLatestDataFile(String partitionPath, String fileId) {
protected Option<HoodieBaseFile> fetchLatestBaseFile(String partitionPath, String fileId) {
// Retries only file-slices of the file and filters for the latest
return Option
.ofNullable(rocksDB
.<HoodieDataFile>prefixSearch(schemaHelper.getColFamilyForView(),
.<HoodieBaseFile>prefixSearch(schemaHelper.getColFamilyForView(),
schemaHelper.getPrefixForDataFileViewByPartitionFile(partitionPath, fileId))
.map(Pair::getValue).reduce(null,
(x, y) -> ((x == null) ? y

View File

@@ -69,8 +69,8 @@ public class CompactionUtils {
builder.setFileId(fileSlice.getFileId());
builder.setBaseInstantTime(fileSlice.getBaseInstantTime());
builder.setDeltaFilePaths(fileSlice.getLogFiles().map(lf -> lf.getPath().getName()).collect(Collectors.toList()));
if (fileSlice.getDataFile().isPresent()) {
builder.setDataFilePath(fileSlice.getDataFile().get().getFileName());
if (fileSlice.getBaseFile().isPresent()) {
builder.setDataFilePath(fileSlice.getBaseFile().get().getFileName());
}
if (metricsCaptureFunction.isPresent()) {