1
0

[HUDI-1746] Added support for replace commits in commit showpartitions, commit show_write_stats, commit showfiles (#2678)

* Added support for replace commits in commit showpartitions, commit show_write_stats, commit showfiles

* Adding CR changes

* [HUDI-1746] Code review changes
This commit is contained in:
jsbali
2021-04-21 23:01:35 +05:30
committed by GitHub
parent b31c520c66
commit 4a3431866d
4 changed files with 282 additions and 30 deletions

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.cli.utils.CommitUtil;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
@@ -34,6 +35,7 @@ import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.NumericUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.spark.launcher.SparkLauncher;
@@ -44,6 +46,7 @@ import org.springframework.stereotype.Component;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@@ -266,15 +269,18 @@ public class CommitsCommand implements CommandMarker {
HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
if (!timeline.containsInstant(commitInstant)) {
Option<HoodieInstant> hoodieInstantOption = getCommitForInstant(timeline, instantTime);
Option<HoodieCommitMetadata> commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption);
if (!commitMetadataOptional.isPresent()) {
return "Commit " + instantTime + " not found in Commits " + timeline;
}
HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(),
HoodieCommitMetadata.class);
HoodieCommitMetadata meta = commitMetadataOptional.get();
List<Comparable[]> rows = new ArrayList<>();
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats().entrySet()) {
String action = hoodieInstantOption.get().getAction();
String path = entry.getKey();
List<HoodieWriteStat> stats = entry.getValue();
long totalFilesAdded = 0;
@@ -294,7 +300,7 @@ public class CommitsCommand implements CommandMarker {
totalBytesWritten += stat.getTotalWriteBytes();
totalWriteErrors += stat.getTotalWriteErrors();
}
rows.add(new Comparable[] {path, totalFilesAdded, totalFilesUpdated, totalRecordsInserted, totalRecordsUpdated,
rows.add(new Comparable[] {action, path, totalFilesAdded, totalFilesUpdated, totalRecordsInserted, totalRecordsUpdated,
totalBytesWritten, totalWriteErrors});
}
@@ -302,7 +308,8 @@ public class CommitsCommand implements CommandMarker {
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN, entry ->
NumericUtils.humanReadableByteCount((Long.parseLong(entry.toString()))));
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_ADDED)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_FILES_UPDATED)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_INSERTED)
@@ -328,24 +335,29 @@ public class CommitsCommand implements CommandMarker {
HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
if (!timeline.containsInstant(commitInstant)) {
Option<HoodieInstant> hoodieInstantOption = getCommitForInstant(timeline, instantTime);
Option<HoodieCommitMetadata> commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption);
if (!commitMetadataOptional.isPresent()) {
return "Commit " + instantTime + " not found in Commits " + timeline;
}
HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(),
HoodieCommitMetadata.class);
HoodieCommitMetadata meta = commitMetadataOptional.get();
String action = hoodieInstantOption.get().getAction();
long recordsWritten = meta.fetchTotalRecordsWritten();
long bytesWritten = meta.fetchTotalBytesWritten();
long avgRecSize = (long) Math.ceil((1.0 * bytesWritten) / recordsWritten);
List<Comparable[]> rows = new ArrayList<>();
rows.add(new Comparable[] {bytesWritten, recordsWritten, avgRecSize});
rows.add(new Comparable[] {action, bytesWritten, recordsWritten, avgRecSize});
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN, entry ->
NumericUtils.humanReadableByteCount((Long.parseLong(entry.toString()))));
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN_COMMIT)
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_BYTES_WRITTEN_COMMIT)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_WRITTEN_COMMIT)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_AVG_REC_SIZE_COMMIT);
@@ -367,24 +379,28 @@ public class CommitsCommand implements CommandMarker {
HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
if (!timeline.containsInstant(commitInstant)) {
Option<HoodieInstant> hoodieInstantOption = getCommitForInstant(timeline, instantTime);
Option<HoodieCommitMetadata> commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption);
if (!commitMetadataOptional.isPresent()) {
return "Commit " + instantTime + " not found in Commits " + timeline;
}
HoodieCommitMetadata meta = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get(),
HoodieCommitMetadata.class);
HoodieCommitMetadata meta = commitMetadataOptional.get();
List<Comparable[]> rows = new ArrayList<>();
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats().entrySet()) {
String action = hoodieInstantOption.get().getAction();
String path = entry.getKey();
List<HoodieWriteStat> stats = entry.getValue();
for (HoodieWriteStat stat : stats) {
rows.add(new Comparable[] {path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(),
rows.add(new Comparable[] {action, path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(),
stat.getNumWrites(), stat.getTotalWriteBytes(), stat.getTotalWriteErrors(), stat.getFileSizeInBytes()});
}
}
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_PREVIOUS_COMMIT)
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_RECORDS_UPDATED)
@@ -431,4 +447,31 @@ public class CommitsCommand implements CommandMarker {
return "Load sync state between " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " and "
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName();
}
/*
Checks whether a commit or replacecommit action exists in the timeline.
* */
private Option<HoodieInstant> getCommitForInstant(HoodieTimeline timeline, String instantTime) throws IOException {
List<HoodieInstant> instants = Arrays.asList(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime),
new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime),
new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime));
Option<HoodieInstant> hoodieInstant = Option.fromJavaOptional(instants.stream().filter(timeline::containsInstant).findAny());
return hoodieInstant;
}
private Option<HoodieCommitMetadata> getHoodieCommitMetadata(HoodieTimeline timeline, Option<HoodieInstant> hoodieInstant) throws IOException {
if (hoodieInstant.isPresent()) {
if (hoodieInstant.get().getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
return Option.of(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get()).get(),
HoodieReplaceCommitMetadata.class));
}
return Option.of(HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get()).get(),
HoodieCommitMetadata.class));
}
return Option.empty();
}
}