[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:
@@ -60,8 +60,8 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
public String showAllFileSlices(
|
||||
@CliOption(key = {"pathRegex"}, help = "regex to select files, eg: 2016/08/02",
|
||||
unspecifiedDefaultValue = "*/*/*") String globRegex,
|
||||
@CliOption(key = {"readOptimizedOnly"}, help = "Only display read-optimized view",
|
||||
unspecifiedDefaultValue = "false") boolean readOptimizedOnly,
|
||||
@CliOption(key = {"baseFileOnly"}, help = "Only display base files view",
|
||||
unspecifiedDefaultValue = "false") boolean baseFileOnly,
|
||||
@CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed",
|
||||
unspecifiedDefaultValue = "") String maxInstant,
|
||||
@CliOption(key = {"includeMax"}, help = "Include Max Instant",
|
||||
@@ -77,19 +77,19 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieTableFileSystemView fsView = buildFileSystemView(globRegex, maxInstant, readOptimizedOnly, includeMaxInstant,
|
||||
HoodieTableFileSystemView fsView = buildFileSystemView(globRegex, maxInstant, baseFileOnly, includeMaxInstant,
|
||||
includeInflight, excludeCompaction);
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
fsView.getAllFileGroups().forEach(fg -> fg.getAllFileSlices().forEach(fs -> {
|
||||
int idx = 0;
|
||||
// For ReadOptimized Views, do not display any delta-file related columns
|
||||
Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 8];
|
||||
// For base file only Views, do not display any delta-file related columns
|
||||
Comparable[] row = new Comparable[baseFileOnly ? 5 : 8];
|
||||
row[idx++] = fg.getPartitionPath();
|
||||
row[idx++] = fg.getFileGroupId().getFileId();
|
||||
row[idx++] = fs.getBaseInstantTime();
|
||||
row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : "";
|
||||
row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1;
|
||||
if (!readOptimizedOnly) {
|
||||
row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getPath() : "";
|
||||
row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getFileSize() : -1;
|
||||
if (!baseFileOnly) {
|
||||
row[idx++] = fs.getLogFiles().count();
|
||||
row[idx++] = fs.getLogFiles().mapToLong(HoodieLogFile::getFileSize).sum();
|
||||
row[idx++] = fs.getLogFiles().collect(Collectors.toList()).toString();
|
||||
@@ -104,7 +104,7 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition").addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant").addTableHeaderField("Data-File").addTableHeaderField("Data-File Size");
|
||||
if (!readOptimizedOnly) {
|
||||
if (!baseFileOnly) {
|
||||
header = header.addTableHeaderField("Num Delta Files").addTableHeaderField("Total Delta File Size")
|
||||
.addTableHeaderField("Delta Files");
|
||||
}
|
||||
@@ -114,8 +114,8 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
@CliCommand(value = "show fsview latest", help = "Show latest file-system view")
|
||||
public String showLatestFileSlices(
|
||||
@CliOption(key = {"partitionPath"}, help = "A valid paritition path", mandatory = true) String partition,
|
||||
@CliOption(key = {"readOptimizedOnly"}, help = "Only display read-optimized view",
|
||||
unspecifiedDefaultValue = "false") boolean readOptimizedOnly,
|
||||
@CliOption(key = {"baseFileOnly"}, help = "Only display base file view",
|
||||
unspecifiedDefaultValue = "false") boolean baseFileOnly,
|
||||
@CliOption(key = {"maxInstant"}, help = "File-Slices upto this instant are displayed",
|
||||
unspecifiedDefaultValue = "") String maxInstant,
|
||||
@CliOption(key = {"merge"}, help = "Merge File Slices due to pending compaction",
|
||||
@@ -133,7 +133,7 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieTableFileSystemView fsView = buildFileSystemView(partition, maxInstant, readOptimizedOnly, includeMaxInstant,
|
||||
HoodieTableFileSystemView fsView = buildFileSystemView(partition, maxInstant, baseFileOnly, includeMaxInstant,
|
||||
includeInflight, excludeCompaction);
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
|
||||
@@ -150,16 +150,16 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
|
||||
fileSliceStream.forEach(fs -> {
|
||||
int idx = 0;
|
||||
Comparable[] row = new Comparable[readOptimizedOnly ? 5 : 13];
|
||||
Comparable[] row = new Comparable[baseFileOnly ? 5 : 13];
|
||||
row[idx++] = partition;
|
||||
row[idx++] = fs.getFileId();
|
||||
row[idx++] = fs.getBaseInstantTime();
|
||||
row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getPath() : "";
|
||||
row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getPath() : "";
|
||||
|
||||
long dataFileSize = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1;
|
||||
long dataFileSize = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getFileSize() : -1;
|
||||
row[idx++] = dataFileSize;
|
||||
|
||||
if (!readOptimizedOnly) {
|
||||
if (!baseFileOnly) {
|
||||
row[idx++] = fs.getLogFiles().count();
|
||||
row[idx++] = fs.getLogFiles().mapToLong(HoodieLogFile::getFileSize).sum();
|
||||
long logFilesScheduledForCompactionTotalSize =
|
||||
@@ -190,7 +190,7 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Data-File Size", converterFunction);
|
||||
if (!readOptimizedOnly) {
|
||||
if (!baseFileOnly) {
|
||||
fieldNameToConverterMap.put("Total Delta Size", converterFunction);
|
||||
fieldNameToConverterMap.put("Delta Size - compaction scheduled", converterFunction);
|
||||
fieldNameToConverterMap.put("Delta Size - compaction unscheduled", converterFunction);
|
||||
@@ -199,7 +199,7 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition").addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant").addTableHeaderField("Data-File").addTableHeaderField("Data-File Size");
|
||||
|
||||
if (!readOptimizedOnly) {
|
||||
if (!baseFileOnly) {
|
||||
header = header.addTableHeaderField("Num Delta Files").addTableHeaderField("Total Delta Size")
|
||||
.addTableHeaderField("Delta Size - compaction scheduled")
|
||||
.addTableHeaderField("Delta Size - compaction unscheduled")
|
||||
@@ -216,14 +216,14 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
*
|
||||
* @param globRegex Path Regex
|
||||
* @param maxInstant Max Instants to be used for displaying file-instants
|
||||
* @param readOptimizedOnly Include only read optimized view
|
||||
* @param basefileOnly Include only base file view
|
||||
* @param includeMaxInstant Include Max instant
|
||||
* @param includeInflight Include inflight instants
|
||||
* @param excludeCompaction Exclude Compaction instants
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private HoodieTableFileSystemView buildFileSystemView(String globRegex, String maxInstant, boolean readOptimizedOnly,
|
||||
private HoodieTableFileSystemView buildFileSystemView(String globRegex, String maxInstant, boolean basefileOnly,
|
||||
boolean includeMaxInstant, boolean includeInflight, boolean excludeCompaction) throws IOException {
|
||||
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
|
||||
HoodieTableMetaClient metaClient =
|
||||
@@ -234,7 +234,7 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
Stream<HoodieInstant> instantsStream;
|
||||
|
||||
HoodieTimeline timeline;
|
||||
if (readOptimizedOnly) {
|
||||
if (basefileOnly) {
|
||||
timeline = metaClient.getActiveTimeline().getCommitTimeline();
|
||||
} else if (excludeCompaction) {
|
||||
timeline = metaClient.getActiveTimeline().getCommitsTimeline();
|
||||
|
||||
@@ -20,7 +20,7 @@ package org.apache.hudi.cli
|
||||
import java.util.stream.Collectors
|
||||
|
||||
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
|
||||
import org.apache.hudi.common.model.{HoodieDataFile, HoodieRecord}
|
||||
import org.apache.hudi.common.model.{HoodieBaseFile, HoodieRecord}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.util.FSUtils
|
||||
@@ -80,7 +80,7 @@ class DedupeSparkJob(basePath: String,
|
||||
|
||||
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"$basePath/$duplicatedPartitionPath"))
|
||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||
val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]())
|
||||
val filteredStatuses = latestFiles.map(f => f.getPath)
|
||||
LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}")
|
||||
|
||||
@@ -132,7 +132,7 @@ class DedupeSparkJob(basePath: String,
|
||||
val allFiles = fs.listStatus(new Path(s"$basePath/$duplicatedPartitionPath"))
|
||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||
|
||||
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||
val latestFiles: java.util.List[HoodieBaseFile] = fsView.getLatestBaseFiles().collect(Collectors.toList[HoodieBaseFile]())
|
||||
|
||||
val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap
|
||||
val dupeFixPlan = planDuplicateFix()
|
||||
|
||||
Reference in New Issue
Block a user