[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:
@@ -37,8 +37,8 @@ public class HoodieHiveUtil {
|
||||
public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp";
|
||||
public static final String HOODIE_MAX_COMMIT_PATTERN = "hoodie.%s.consume.max.commits";
|
||||
public static final String INCREMENTAL_SCAN_MODE = "INCREMENTAL";
|
||||
public static final String LATEST_SCAN_MODE = "LATEST";
|
||||
public static final String DEFAULT_SCAN_MODE = LATEST_SCAN_MODE;
|
||||
public static final String SNAPSHOT_SCAN_MODE = "SNAPSHOT";
|
||||
public static final String DEFAULT_SCAN_MODE = SNAPSHOT_SCAN_MODE;
|
||||
public static final int DEFAULT_MAX_COMMITS = 1;
|
||||
public static final int MAX_COMMIT_ALL = -1;
|
||||
public static final int DEFAULT_LEVELS_TO_BASEPATH = 3;
|
||||
@@ -60,13 +60,6 @@ public class HoodieHiveUtil {
|
||||
return job.getConfiguration().get(startCommitTimestampName);
|
||||
}
|
||||
|
||||
public static String readMode(JobContext job, String tableName) {
|
||||
String modePropertyName = String.format(HOODIE_CONSUME_MODE_PATTERN, tableName);
|
||||
String mode = job.getConfiguration().get(modePropertyName, DEFAULT_SCAN_MODE);
|
||||
LOG.info(modePropertyName + ": " + mode);
|
||||
return mode;
|
||||
}
|
||||
|
||||
public static Path getNthParent(Path path, int n) {
|
||||
Path parent = path;
|
||||
for (int i = 0; i < n; i++) {
|
||||
|
||||
@@ -41,11 +41,11 @@ import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapred.Reporter;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
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.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
@@ -177,12 +177,12 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
// Mutate the JobConf to set the input paths to only partitions touched by incremental pull.
|
||||
setInputPaths(job, incrementalInputPaths);
|
||||
FileStatus[] fileStatuses = super.listStatus(job);
|
||||
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(tableMetaClient, timeline,
|
||||
BaseFileOnlyView roView = new HoodieTableFileSystemView(tableMetaClient, timeline,
|
||||
fileStatuses);
|
||||
List<String> commitsList = commitsToCheck.stream().map(s -> s.getTimestamp()).collect(Collectors.toList());
|
||||
List<HoodieDataFile> filteredFiles = roView.getLatestDataFilesInRange(commitsList).collect(Collectors.toList());
|
||||
List<HoodieBaseFile> filteredFiles = roView.getLatestBaseFilesInRange(commitsList).collect(Collectors.toList());
|
||||
List<FileStatus> returns = new ArrayList<>();
|
||||
for (HoodieDataFile filteredFile : filteredFiles) {
|
||||
for (HoodieBaseFile filteredFile : filteredFiles) {
|
||||
LOG.debug("Processing incremental hoodie file - " + filteredFile.getPath());
|
||||
filteredFile = checkFileStatus(filteredFile);
|
||||
returns.add(filteredFile.getFileStatus());
|
||||
@@ -238,12 +238,12 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
}
|
||||
// Get all commits, delta commits, compactions, as all of them produce a base parquet file today
|
||||
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metadata, timeline, statuses);
|
||||
BaseFileOnlyView roView = new HoodieTableFileSystemView(metadata, timeline, statuses);
|
||||
// filter files on the latest commit found
|
||||
List<HoodieDataFile> filteredFiles = roView.getLatestDataFiles().collect(Collectors.toList());
|
||||
List<HoodieBaseFile> filteredFiles = roView.getLatestBaseFiles().collect(Collectors.toList());
|
||||
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
|
||||
List<FileStatus> returns = new ArrayList<>();
|
||||
for (HoodieDataFile filteredFile : filteredFiles) {
|
||||
for (HoodieBaseFile filteredFile : filteredFiles) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Processing latest hoodie file - " + filteredFile.getPath());
|
||||
}
|
||||
@@ -258,13 +258,13 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
* super.listStatus() and gets back a FileStatus[] 2. Then it creates the HoodieTableMetaClient for the paths listed.
|
||||
* 3. Generation of splits looks at FileStatus size to create splits, which skips this file
|
||||
*/
|
||||
private HoodieDataFile checkFileStatus(HoodieDataFile dataFile) throws IOException {
|
||||
private HoodieBaseFile checkFileStatus(HoodieBaseFile dataFile) throws IOException {
|
||||
Path dataPath = dataFile.getFileStatus().getPath();
|
||||
try {
|
||||
if (dataFile.getFileSize() == 0) {
|
||||
FileSystem fs = dataPath.getFileSystem(conf);
|
||||
LOG.info("Refreshing file status " + dataFile.getPath());
|
||||
return new HoodieDataFile(fs.getFileStatus(dataPath));
|
||||
return new HoodieBaseFile(fs.getFileStatus(dataPath));
|
||||
}
|
||||
return dataFile;
|
||||
} catch (IOException e) {
|
||||
|
||||
@@ -18,7 +18,7 @@
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
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.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
@@ -139,14 +139,14 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), fs.listStatus(folder));
|
||||
List<HoodieDataFile> latestFiles = fsView.getLatestDataFiles().collect(Collectors.toList());
|
||||
List<HoodieBaseFile> latestFiles = fsView.getLatestBaseFiles().collect(Collectors.toList());
|
||||
// populate the cache
|
||||
if (!hoodiePathCache.containsKey(folder.toString())) {
|
||||
hoodiePathCache.put(folder.toString(), new HashSet<>());
|
||||
}
|
||||
LOG.info("Based on hoodie metadata from base path: " + baseDir.toString() + ", caching " + latestFiles.size()
|
||||
+ " files under " + folder);
|
||||
for (HoodieDataFile lfile : latestFiles) {
|
||||
for (HoodieBaseFile lfile : latestFiles) {
|
||||
hoodiePathCache.get(folder.toString()).add(new Path(lfile.getPath()));
|
||||
}
|
||||
|
||||
|
||||
@@ -336,7 +336,7 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
*/
|
||||
private void init() throws IOException {
|
||||
Schema schemaFromLogFile =
|
||||
LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaFilePaths(), jobConf);
|
||||
LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaLogPaths(), jobConf);
|
||||
if (schemaFromLogFile == null) {
|
||||
writerSchema = new AvroSchemaConverter().convert(baseFileSchema);
|
||||
LOG.debug("Writer Schema From Parquet => " + writerSchema.getFields());
|
||||
@@ -360,7 +360,7 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
|
||||
readerSchema = generateProjectionSchema(writerSchema, schemaFieldsMap, projectionFields);
|
||||
LOG.info(String.format("About to read compacted logs %s for base split %s, projecting cols %s",
|
||||
split.getDeltaFilePaths(), split.getPath(), projectionFields));
|
||||
split.getDeltaLogPaths(), split.getPath(), projectionFields));
|
||||
}
|
||||
|
||||
private Schema constructHiveOrderedSchema(Schema writerSchema, Map<String, Field> schemaFieldsMap) {
|
||||
|
||||
@@ -32,7 +32,7 @@ import java.util.List;
|
||||
*/
|
||||
public class HoodieRealtimeFileSplit extends FileSplit {
|
||||
|
||||
private List<String> deltaFilePaths;
|
||||
private List<String> deltaLogPaths;
|
||||
|
||||
private String maxCommitTime;
|
||||
|
||||
@@ -42,16 +42,16 @@ public class HoodieRealtimeFileSplit extends FileSplit {
|
||||
super();
|
||||
}
|
||||
|
||||
public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List<String> deltaLogFiles, String maxCommitTime)
|
||||
public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List<String> deltaLogPaths, String maxCommitTime)
|
||||
throws IOException {
|
||||
super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations());
|
||||
this.deltaFilePaths = deltaLogFiles;
|
||||
this.deltaLogPaths = deltaLogPaths;
|
||||
this.maxCommitTime = maxCommitTime;
|
||||
this.basePath = basePath;
|
||||
}
|
||||
|
||||
public List<String> getDeltaFilePaths() {
|
||||
return deltaFilePaths;
|
||||
public List<String> getDeltaLogPaths() {
|
||||
return deltaLogPaths;
|
||||
}
|
||||
|
||||
public String getMaxCommitTime() {
|
||||
@@ -79,8 +79,8 @@ public class HoodieRealtimeFileSplit extends FileSplit {
|
||||
super.write(out);
|
||||
writeString(basePath, out);
|
||||
writeString(maxCommitTime, out);
|
||||
out.writeInt(deltaFilePaths.size());
|
||||
for (String logFilePath : deltaFilePaths) {
|
||||
out.writeInt(deltaLogPaths.size());
|
||||
for (String logFilePath : deltaLogPaths) {
|
||||
writeString(logFilePath, out);
|
||||
}
|
||||
}
|
||||
@@ -91,15 +91,15 @@ public class HoodieRealtimeFileSplit extends FileSplit {
|
||||
basePath = readString(in);
|
||||
maxCommitTime = readString(in);
|
||||
int totalLogFiles = in.readInt();
|
||||
deltaFilePaths = new ArrayList<>(totalLogFiles);
|
||||
deltaLogPaths = new ArrayList<>(totalLogFiles);
|
||||
for (int i = 0; i < totalLogFiles; i++) {
|
||||
deltaFilePaths.add(readString(in));
|
||||
deltaLogPaths.add(readString(in));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HoodieRealtimeFileSplit{DataPath=" + getPath() + ", deltaFilePaths=" + deltaFilePaths
|
||||
return "HoodieRealtimeFileSplit{DataPath=" + getPath() + ", deltaLogPaths=" + deltaLogPaths
|
||||
+ ", maxCommitTime='" + maxCommitTime + '\'' + ", basePath='" + basePath + '\'' + '}';
|
||||
}
|
||||
}
|
||||
|
||||
@@ -61,7 +61,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
|
||||
// but can return records for completed commits > the commit we are trying to read (if using
|
||||
// readCommit() API)
|
||||
return new HoodieMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf), split.getBasePath(),
|
||||
split.getDeltaFilePaths(), usesCustomPayload ? getWriterSchema() : getReaderSchema(), split.getMaxCommitTime(),
|
||||
split.getDeltaLogPaths(), usesCustomPayload ? getWriterSchema() : getReaderSchema(), split.getMaxCommitTime(),
|
||||
getMaxCompactionMemoryInBytes(),
|
||||
Boolean
|
||||
.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
|
||||
@@ -77,7 +77,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
|
||||
// Consumer of this record reader
|
||||
this.iterator = this.executor.getQueue().iterator();
|
||||
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
|
||||
split.getBasePath(), split.getDeltaFilePaths(), getReaderSchema(), split.getMaxCommitTime(),
|
||||
split.getBasePath(), split.getDeltaLogPaths(), getReaderSchema(), split.getMaxCommitTime(),
|
||||
Boolean
|
||||
.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), record -> {
|
||||
|
||||
Reference in New Issue
Block a user