[HUDI-296] Explore use of spotless to auto fix formatting errors (#945)
- Add spotless format fixing to project - One time reformatting for conformity - Build fails for formatting changes and mvn spotless:apply autofixes them
This commit is contained in:
@@ -52,19 +52,16 @@ public class HoodiePrintHelper {
|
||||
* @param rows List of rows
|
||||
* @return Serialized form for printing
|
||||
*/
|
||||
public static String print(TableHeader rowHeader,
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap,
|
||||
String sortByField, boolean isDescending, Integer limit, boolean headerOnly,
|
||||
List<Comparable[]> rows) {
|
||||
public static String print(TableHeader rowHeader, Map<String, Function<Object, String>> fieldNameToConverterMap,
|
||||
String sortByField, boolean isDescending, Integer limit, boolean headerOnly, List<Comparable[]> rows) {
|
||||
|
||||
if (headerOnly) {
|
||||
return HoodiePrintHelper.print(rowHeader);
|
||||
}
|
||||
|
||||
Table table = new Table(rowHeader, fieldNameToConverterMap,
|
||||
Option.ofNullable(sortByField.isEmpty() ? null : sortByField),
|
||||
Option.ofNullable(isDescending),
|
||||
Option.ofNullable(limit <= 0 ? null : limit)).addAllRows(rows).flip();
|
||||
Table table =
|
||||
new Table(rowHeader, fieldNameToConverterMap, Option.ofNullable(sortByField.isEmpty() ? null : sortByField),
|
||||
Option.ofNullable(isDescending), Option.ofNullable(limit <= 0 ? null : limit)).addAllRows(rows).flip();
|
||||
|
||||
return HoodiePrintHelper.print(table);
|
||||
}
|
||||
@@ -79,9 +76,8 @@ public class HoodiePrintHelper {
|
||||
String[] header = new String[buffer.getFieldNames().size()];
|
||||
buffer.getFieldNames().toArray(header);
|
||||
|
||||
String[][] rows = buffer.getRenderRows().stream()
|
||||
.map(l -> l.stream().toArray(String[]::new))
|
||||
.toArray(String[][]::new);
|
||||
String[][] rows =
|
||||
buffer.getRenderRows().stream().map(l -> l.stream().toArray(String[]::new)).toArray(String[][]::new);
|
||||
return printTextTable(header, rows);
|
||||
}
|
||||
|
||||
@@ -94,7 +90,7 @@ public class HoodiePrintHelper {
|
||||
private static String print(TableHeader header) {
|
||||
String[] head = new String[header.getFieldNames().size()];
|
||||
header.getFieldNames().toArray(head);
|
||||
return printTextTable(head, new String[][]{});
|
||||
return printTextTable(head, new String[][] {});
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -31,8 +31,7 @@ import java.util.stream.IntStream;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Table to be rendered. This class takes care of ordering
|
||||
* rows and limiting before renderer renders it.
|
||||
* Table to be rendered. This class takes care of ordering rows and limiting before renderer renders it.
|
||||
*/
|
||||
public class Table implements Iterable<List<String>> {
|
||||
|
||||
@@ -53,11 +52,8 @@ public class Table implements Iterable<List<String>> {
|
||||
// Rows ready for Rendering
|
||||
private List<List<String>> renderRows;
|
||||
|
||||
public Table(TableHeader rowHeader,
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap,
|
||||
Option<String> orderingFieldNameOptional,
|
||||
Option<Boolean> isDescendingOptional,
|
||||
Option<Integer> limitOptional) {
|
||||
public Table(TableHeader rowHeader, Map<String, Function<Object, String>> fieldNameToConverterMap,
|
||||
Option<String> orderingFieldNameOptional, Option<Boolean> isDescendingOptional, Option<Integer> limitOptional) {
|
||||
this.rowHeader = rowHeader;
|
||||
this.fieldNameToConverterMap = fieldNameToConverterMap;
|
||||
this.orderingFieldNameOptional = orderingFieldNameOptional;
|
||||
@@ -68,6 +64,7 @@ public class Table implements Iterable<List<String>> {
|
||||
|
||||
/**
|
||||
* Main API to add row to the table
|
||||
*
|
||||
* @param row Row
|
||||
*/
|
||||
public Table add(List<Comparable> row) {
|
||||
@@ -86,6 +83,7 @@ public class Table implements Iterable<List<String>> {
|
||||
|
||||
/**
|
||||
* Add all rows
|
||||
*
|
||||
* @param rows Rows to be aded
|
||||
* @return
|
||||
*/
|
||||
@@ -96,6 +94,7 @@ public class Table implements Iterable<List<String>> {
|
||||
|
||||
/**
|
||||
* Add all rows
|
||||
*
|
||||
* @param rows Rows to be added
|
||||
* @return
|
||||
*/
|
||||
@@ -115,6 +114,7 @@ public class Table implements Iterable<List<String>> {
|
||||
|
||||
/**
|
||||
* Sorting of rows by a specified field
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
private List<List<Comparable>> orderRows() {
|
||||
|
||||
@@ -59,8 +59,8 @@ public class ArchivedCommitsCommand implements CommandMarker {
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
System.out.println("===============> Showing only " + limit + " archived commits <===============");
|
||||
String basePath = HoodieCLI.tableMetadata.getBasePath();
|
||||
@@ -71,12 +71,12 @@ public class ArchivedCommitsCommand implements CommandMarker {
|
||||
FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
|
||||
List<Comparable[]> allStats = new ArrayList<>();
|
||||
for (FileStatus fs : fsStatuses) {
|
||||
//read the archived file
|
||||
// read the archived file
|
||||
Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
|
||||
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
|
||||
|
||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||
//read the avro blocks
|
||||
// read the avro blocks
|
||||
while (reader.hasNext()) {
|
||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||
List<IndexedRecord> records = blk.getRecords();
|
||||
@@ -86,9 +86,8 @@ public class ArchivedCommitsCommand implements CommandMarker {
|
||||
.filter(r -> r.get("actionType").toString().equals(HoodieTimeline.COMMIT_ACTION)
|
||||
|| r.get("actionType").toString().equals(HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.flatMap(r -> {
|
||||
HoodieCommitMetadata metadata =
|
||||
(HoodieCommitMetadata) SpecificData.get().deepCopy(HoodieCommitMetadata.SCHEMA$,
|
||||
r.get("hoodieCommitMetadata"));
|
||||
HoodieCommitMetadata metadata = (HoodieCommitMetadata) SpecificData.get()
|
||||
.deepCopy(HoodieCommitMetadata.SCHEMA$, r.get("hoodieCommitMetadata"));
|
||||
final String instantTime = r.get("commitTime").toString();
|
||||
final String action = r.get("actionType").toString();
|
||||
return metadata.getPartitionToWriteStats().values().stream().flatMap(hoodieWriteStats -> {
|
||||
@@ -118,22 +117,13 @@ public class ArchivedCommitsCommand implements CommandMarker {
|
||||
allStats.addAll(readCommits);
|
||||
reader.close();
|
||||
}
|
||||
TableHeader header = new TableHeader().addTableHeaderField("action")
|
||||
.addTableHeaderField("instant")
|
||||
.addTableHeaderField("partition")
|
||||
.addTableHeaderField("file_id")
|
||||
.addTableHeaderField("prev_instant")
|
||||
.addTableHeaderField("num_writes")
|
||||
.addTableHeaderField("num_inserts")
|
||||
.addTableHeaderField("num_deletes")
|
||||
.addTableHeaderField("num_update_writes")
|
||||
.addTableHeaderField("total_log_files")
|
||||
.addTableHeaderField("total_log_blocks")
|
||||
.addTableHeaderField("total_corrupt_log_blocks")
|
||||
.addTableHeaderField("total_rollback_blocks")
|
||||
.addTableHeaderField("total_log_records")
|
||||
.addTableHeaderField("total_updated_records_compacted")
|
||||
.addTableHeaderField("total_write_bytes")
|
||||
TableHeader header = new TableHeader().addTableHeaderField("action").addTableHeaderField("instant")
|
||||
.addTableHeaderField("partition").addTableHeaderField("file_id").addTableHeaderField("prev_instant")
|
||||
.addTableHeaderField("num_writes").addTableHeaderField("num_inserts").addTableHeaderField("num_deletes")
|
||||
.addTableHeaderField("num_update_writes").addTableHeaderField("total_log_files")
|
||||
.addTableHeaderField("total_log_blocks").addTableHeaderField("total_corrupt_log_blocks")
|
||||
.addTableHeaderField("total_rollback_blocks").addTableHeaderField("total_log_records")
|
||||
.addTableHeaderField("total_updated_records_compacted").addTableHeaderField("total_write_bytes")
|
||||
.addTableHeaderField("total_write_errors");
|
||||
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, allStats);
|
||||
@@ -141,41 +131,39 @@ public class ArchivedCommitsCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "show archived commits", help = "Read commits from archived files and show details")
|
||||
public String showCommits(
|
||||
@CliOption(key = {"skipMetadata"}, help = "Skip displaying commit metadata", unspecifiedDefaultValue = "true")
|
||||
boolean skipMetadata,
|
||||
@CliOption(key = {"skipMetadata"}, help = "Skip displaying commit metadata",
|
||||
unspecifiedDefaultValue = "true") boolean skipMetadata,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
System.out.println("===============> Showing only " + limit + " archived commits <===============");
|
||||
String basePath = HoodieCLI.tableMetadata.getBasePath();
|
||||
FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf)
|
||||
.globStatus(new Path(basePath + "/.hoodie/.commits_.archive*"));
|
||||
FileStatus[] fsStatuses =
|
||||
FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(new Path(basePath + "/.hoodie/.commits_.archive*"));
|
||||
List<Comparable[]> allCommits = new ArrayList<>();
|
||||
for (FileStatus fs : fsStatuses) {
|
||||
//read the archived file
|
||||
// read the archived file
|
||||
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
|
||||
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
|
||||
|
||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||
//read the avro blocks
|
||||
// read the avro blocks
|
||||
while (reader.hasNext()) {
|
||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||
List<IndexedRecord> records = blk.getRecords();
|
||||
readRecords.addAll(records);
|
||||
}
|
||||
List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r ->
|
||||
readCommit(r, skipMetadata))
|
||||
.collect(Collectors.toList());
|
||||
List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r)
|
||||
.map(r -> readCommit(r, skipMetadata)).collect(Collectors.toList());
|
||||
allCommits.addAll(readCommits);
|
||||
reader.close();
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader().addTableHeaderField("CommitTime")
|
||||
.addTableHeaderField("CommitType");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("CommitTime").addTableHeaderField("CommitType");
|
||||
|
||||
if (!skipMetadata) {
|
||||
header = header.addTableHeaderField("CommitDetails");
|
||||
|
||||
@@ -63,8 +63,8 @@ public class CleansCommand implements CommandMarker {
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
@@ -74,17 +74,15 @@ public class CleansCommand implements CommandMarker {
|
||||
Collections.reverse(cleans);
|
||||
for (int i = 0; i < cleans.size(); i++) {
|
||||
HoodieInstant clean = cleans.get(i);
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils
|
||||
.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get());
|
||||
rows.add(new Comparable[]{clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
|
||||
HoodieCleanMetadata cleanMetadata =
|
||||
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get());
|
||||
rows.add(new Comparable[] {clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
|
||||
cleanMetadata.getTotalFilesDeleted(), cleanMetadata.getTimeTakenInMillis()});
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("CleanTime")
|
||||
.addTableHeaderField("EarliestCommandRetained")
|
||||
.addTableHeaderField("Total Files Deleted")
|
||||
.addTableHeaderField("Total Time Taken");
|
||||
TableHeader header =
|
||||
new TableHeader().addTableHeaderField("CleanTime").addTableHeaderField("EarliestCommandRetained")
|
||||
.addTableHeaderField("Total Files Deleted").addTableHeaderField("Total Time Taken");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@@ -95,13 +93,12 @@ public class CleansCommand implements CommandMarker {
|
||||
}
|
||||
|
||||
@CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean")
|
||||
public String showCleanPartitions(
|
||||
@CliOption(key = {"clean"}, help = "clean to show") final String commitTime,
|
||||
public String showCleanPartitions(@CliOption(key = {"clean"}, help = "clean to show") final String commitTime,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
@@ -112,8 +109,8 @@ public class CleansCommand implements CommandMarker {
|
||||
return "Clean " + commitTime + " not found in metadata " + timeline;
|
||||
}
|
||||
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata(
|
||||
timeline.getInstantDetails(cleanInstant).get());
|
||||
HoodieCleanMetadata cleanMetadata =
|
||||
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(cleanInstant).get());
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
for (Map.Entry<String, HoodieCleanPartitionMetadata> entry : cleanMetadata.getPartitionMetadata().entrySet()) {
|
||||
String path = entry.getKey();
|
||||
@@ -121,14 +118,11 @@ public class CleansCommand implements CommandMarker {
|
||||
String policy = stats.getPolicy();
|
||||
Integer totalSuccessDeletedFiles = stats.getSuccessDeleteFiles().size();
|
||||
Integer totalFailedDeletedFiles = stats.getFailedDeleteFiles().size();
|
||||
rows.add(new Comparable[]{path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles});
|
||||
rows.add(new Comparable[] {path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles});
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("Cleaning policy")
|
||||
.addTableHeaderField("Total Files Successfully Deleted")
|
||||
.addTableHeaderField("Total Failed Deletions");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition Path").addTableHeaderField("Cleaning policy")
|
||||
.addTableHeaderField("Total Files Successfully Deleted").addTableHeaderField("Total Failed Deletions");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
|
||||
}
|
||||
|
||||
@@ -69,12 +69,13 @@ public class CommitsCommand implements CommandMarker {
|
||||
}
|
||||
|
||||
@CliCommand(value = "commits show", help = "Show the commits")
|
||||
public String showCommits(@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
public String showCommits(
|
||||
@CliOption(key = {"limit"}, mandatory = false, help = "Limit commits",
|
||||
unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
@@ -84,16 +85,12 @@ public class CommitsCommand implements CommandMarker {
|
||||
Collections.reverse(commits);
|
||||
for (int i = 0; i < commits.size(); i++) {
|
||||
HoodieInstant commit = commits.get(i);
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
rows.add(new Comparable[]{commit.getTimestamp(),
|
||||
commitMetadata.fetchTotalBytesWritten(),
|
||||
commitMetadata.fetchTotalFilesInsert(),
|
||||
commitMetadata.fetchTotalFilesUpdated(),
|
||||
commitMetadata.fetchTotalPartitionsWritten(),
|
||||
commitMetadata.fetchTotalRecordsWritten(),
|
||||
commitMetadata.fetchTotalUpdateRecordsWritten(),
|
||||
commitMetadata.fetchTotalWriteErrors()});
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class);
|
||||
rows.add(new Comparable[] {commit.getTimestamp(), commitMetadata.fetchTotalBytesWritten(),
|
||||
commitMetadata.fetchTotalFilesInsert(), commitMetadata.fetchTotalFilesUpdated(),
|
||||
commitMetadata.fetchTotalPartitionsWritten(), commitMetadata.fetchTotalRecordsWritten(),
|
||||
commitMetadata.fetchTotalUpdateRecordsWritten(), commitMetadata.fetchTotalWriteErrors()});
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
@@ -101,15 +98,10 @@ public class CommitsCommand implements CommandMarker {
|
||||
return NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
});
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("CommitTime")
|
||||
.addTableHeaderField("Total Bytes Written")
|
||||
.addTableHeaderField("Total Files Added")
|
||||
.addTableHeaderField("Total Files Updated")
|
||||
.addTableHeaderField("Total Partitions Written")
|
||||
.addTableHeaderField("Total Records Written")
|
||||
.addTableHeaderField("Total Update Records Written")
|
||||
.addTableHeaderField("Total Errors");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("CommitTime").addTableHeaderField("Total Bytes Written")
|
||||
.addTableHeaderField("Total Files Added").addTableHeaderField("Total Files Updated")
|
||||
.addTableHeaderField("Total Partitions Written").addTableHeaderField("Total Records Written")
|
||||
.addTableHeaderField("Total Update Records Written").addTableHeaderField("Total Errors");
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@@ -132,8 +124,8 @@ public class CommitsCommand implements CommandMarker {
|
||||
}
|
||||
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher
|
||||
.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), commitTime, HoodieCLI.tableMetadata.getBasePath());
|
||||
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), commitTime,
|
||||
HoodieCLI.tableMetadata.getBasePath());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
@@ -146,13 +138,12 @@ public class CommitsCommand implements CommandMarker {
|
||||
}
|
||||
|
||||
@CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit")
|
||||
public String showCommitPartitions(
|
||||
@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime,
|
||||
public String showCommitPartitions(@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
@@ -185,8 +176,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[] {path, totalFilesAdded, totalFilesUpdated, totalRecordsInserted, totalRecordsUpdated,
|
||||
totalBytesWritten, totalWriteErrors});
|
||||
}
|
||||
|
||||
@@ -195,26 +185,21 @@ public class CommitsCommand implements CommandMarker {
|
||||
return NumericUtils.humanReadableByteCount((Long.valueOf(entry.toString())));
|
||||
});
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("Total Files Added")
|
||||
.addTableHeaderField("Total Files Updated")
|
||||
.addTableHeaderField("Total Records Inserted")
|
||||
.addTableHeaderField("Total Records Updated")
|
||||
.addTableHeaderField("Total Bytes Written")
|
||||
.addTableHeaderField("Total Errors");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("Total Files Added").addTableHeaderField("Total Files Updated")
|
||||
.addTableHeaderField("Total Records Inserted").addTableHeaderField("Total Records Updated")
|
||||
.addTableHeaderField("Total Bytes Written").addTableHeaderField("Total Errors");
|
||||
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "commit showfiles", help = "Show file level details of a commit")
|
||||
public String showCommitFiles(
|
||||
@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime,
|
||||
public String showCommitFiles(@CliOption(key = {"commit"}, help = "Commit to show") final String commitTime,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
@@ -231,23 +216,15 @@ public class CommitsCommand implements CommandMarker {
|
||||
String path = entry.getKey();
|
||||
List<HoodieWriteStat> stats = entry.getValue();
|
||||
for (HoodieWriteStat stat : stats) {
|
||||
rows.add(new Comparable[]{path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(),
|
||||
stat.getNumWrites(), stat.getTotalWriteBytes(),
|
||||
stat.getTotalWriteErrors(),
|
||||
stat.getFileSizeInBytes()
|
||||
});
|
||||
rows.add(new Comparable[] {path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(),
|
||||
stat.getNumWrites(), stat.getTotalWriteBytes(), stat.getTotalWriteErrors(), stat.getFileSizeInBytes()});
|
||||
}
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("File ID")
|
||||
.addTableHeaderField("Previous Commit")
|
||||
.addTableHeaderField("Total Records Updated")
|
||||
.addTableHeaderField("Total Records Written")
|
||||
.addTableHeaderField("Total Bytes Written")
|
||||
.addTableHeaderField("Total Errors")
|
||||
.addTableHeaderField("File Size");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition Path").addTableHeaderField("File ID")
|
||||
.addTableHeaderField("Previous Commit").addTableHeaderField("Total Records Updated")
|
||||
.addTableHeaderField("Total Records Written").addTableHeaderField("Total Bytes Written")
|
||||
.addTableHeaderField("Total Errors").addTableHeaderField("File Size");
|
||||
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
@@ -270,8 +247,8 @@ public class CommitsCommand implements CommandMarker {
|
||||
String sourceLatestCommit =
|
||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp();
|
||||
|
||||
if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit,
|
||||
HoodieTimeline.GREATER)) {
|
||||
if (sourceLatestCommit != null
|
||||
&& HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
||||
// source is behind the target
|
||||
List<String> commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
|
||||
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
|
||||
@@ -75,16 +75,15 @@ public class CompactionCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "compactions show all", help = "Shows all compactions that are in active timeline")
|
||||
public String compactionsAll(
|
||||
@CliOption(key = {
|
||||
"includeExtraMetadata"}, help = "Include extra metadata", unspecifiedDefaultValue = "false") final
|
||||
boolean includeExtraMetadata,
|
||||
@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"includeExtraMetadata"}, help = "Include extra metadata",
|
||||
unspecifiedDefaultValue = "false") final boolean includeExtraMetadata,
|
||||
@CliOption(key = {"limit"}, mandatory = false, help = "Limit commits",
|
||||
unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final
|
||||
boolean headerOnly) throws IOException {
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionTimeline();
|
||||
HoodieTimeline commitTimeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
@@ -99,15 +98,14 @@ public class CompactionCommand implements CommandMarker {
|
||||
if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
try {
|
||||
// This could be a completed compaction. Assume a compaction request file is present but skip if fails
|
||||
workload = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
} catch (HoodieIOException ioe) {
|
||||
// SKIP
|
||||
}
|
||||
} else {
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
}
|
||||
|
||||
if (null != workload) {
|
||||
@@ -116,22 +114,18 @@ public class CompactionCommand implements CommandMarker {
|
||||
state = State.COMPLETED;
|
||||
}
|
||||
if (includeExtraMetadata) {
|
||||
rows.add(new Comparable[]{instant.getTimestamp(),
|
||||
state.toString(),
|
||||
rows.add(new Comparable[] {instant.getTimestamp(), state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size(),
|
||||
workload.getExtraMetadata().toString()});
|
||||
} else {
|
||||
rows.add(new Comparable[]{instant.getTimestamp(),
|
||||
state.toString(),
|
||||
rows.add(new Comparable[] {instant.getTimestamp(), state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size()});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Compaction Instant Time")
|
||||
.addTableHeaderField("State")
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Compaction Instant Time").addTableHeaderField("State")
|
||||
.addTableHeaderField("Total FileIds to be Compacted");
|
||||
if (includeExtraMetadata) {
|
||||
header = header.addTableHeaderField("Extra Metadata");
|
||||
@@ -141,48 +135,37 @@ public class CompactionCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "compaction show", help = "Shows compaction details for a specific compaction instant")
|
||||
public String compactionShow(
|
||||
@CliOption(key = "instant", mandatory = true, help = "Base path for the target hoodie dataset") final
|
||||
String compactionInstantTime,
|
||||
@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = "instant", mandatory = true,
|
||||
help = "Base path for the target hoodie dataset") final String compactionInstantTime,
|
||||
@CliOption(key = {"limit"}, mandatory = false, help = "Limit commits",
|
||||
unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
if ((null != workload) && (null != workload.getOperations())) {
|
||||
for (HoodieCompactionOperation op : workload.getOperations()) {
|
||||
rows.add(new Comparable[]{op.getPartitionPath(),
|
||||
op.getFileId(),
|
||||
op.getBaseInstantTime(),
|
||||
op.getDataFilePath(),
|
||||
op.getDeltaFilePaths().size(),
|
||||
op.getMetrics() == null ? "" : op.getMetrics().toString()
|
||||
});
|
||||
rows.add(new Comparable[] {op.getPartitionPath(), op.getFileId(), op.getBaseInstantTime(), op.getDataFilePath(),
|
||||
op.getDeltaFilePaths().size(), op.getMetrics() == null ? "" : op.getMetrics().toString()});
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition Path")
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Base Instant")
|
||||
.addTableHeaderField("Data File Path")
|
||||
.addTableHeaderField("Total Delta Files")
|
||||
.addTableHeaderField("getMetrics");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition Path").addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Base Instant").addTableHeaderField("Data File Path")
|
||||
.addTableHeaderField("Total Delta Files").addTableHeaderField("getMetrics");
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "compaction schedule", help = "Schedule Compaction")
|
||||
public String scheduleCompact(
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "1G", help = "Spark executor memory")
|
||||
final String sparkMemory) throws Exception {
|
||||
public String scheduleCompact(@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "1G",
|
||||
help = "Spark executor memory") final String sparkMemory) throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
@@ -190,8 +173,8 @@ public class CompactionCommand implements CommandMarker {
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
String sparkPropertiesPath =
|
||||
Utils.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_SCHEDULE.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
HoodieCLI.tableMetadata.getTableConfig().getTableName(), compactionInstantTime, sparkMemory);
|
||||
@@ -209,33 +192,34 @@ public class CompactionCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "compaction run", help = "Run Compaction for given instant time")
|
||||
public String compact(
|
||||
@CliOption(key = {"parallelism"}, mandatory = true, help = "Parallelism for hoodie compaction")
|
||||
final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file")
|
||||
final String schemaFilePath,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G", help = "Spark executor memory")
|
||||
final String sparkMemory,
|
||||
@CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries")
|
||||
final String retry,
|
||||
@CliOption(key = "compactionInstant", mandatory = false, help = "Base path for the target hoodie dataset")
|
||||
String compactionInstantTime) throws Exception {
|
||||
@CliOption(key = {"parallelism"}, mandatory = true,
|
||||
help = "Parallelism for hoodie compaction") final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true,
|
||||
help = "Path for Avro schema file") final String schemaFilePath,
|
||||
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
|
||||
help = "Spark executor memory") final String sparkMemory,
|
||||
@CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries") final String retry,
|
||||
@CliOption(key = "compactionInstant", mandatory = false,
|
||||
help = "Base path for the target hoodie dataset") String compactionInstantTime)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
if (null == compactionInstantTime) {
|
||||
// pick outstanding one with lowest timestamp
|
||||
Option<String> firstPendingInstant = HoodieCLI.tableMetadata.reloadActiveTimeline()
|
||||
.filterCompletedAndCompactionInstants().filter(instant -> instant.getAction()
|
||||
.equals(HoodieTimeline.COMPACTION_ACTION)).firstInstant().map(HoodieInstant::getTimestamp);
|
||||
Option<String> firstPendingInstant =
|
||||
HoodieCLI.tableMetadata.reloadActiveTimeline().filterCompletedAndCompactionInstants()
|
||||
.filter(instant -> instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)).firstInstant()
|
||||
.map(HoodieInstant::getTimestamp);
|
||||
if (!firstPendingInstant.isPresent()) {
|
||||
return "NO PENDING COMPACTION TO RUN";
|
||||
}
|
||||
compactionInstantTime = firstPendingInstant.get();
|
||||
}
|
||||
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
String sparkPropertiesPath =
|
||||
Utils.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_RUN.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
HoodieCLI.tableMetadata.getTableConfig().getTableName(), compactionInstantTime, parallelism, schemaFilePath,
|
||||
@@ -279,8 +263,8 @@ public class CompactionCommand implements CommandMarker {
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
@@ -290,12 +274,11 @@ public class CompactionCommand implements CommandMarker {
|
||||
String output = null;
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
String sparkPropertiesPath = Utils
|
||||
.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_VALIDATE.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_VALIDATE.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
compactionInstant, outputPathStr, parallelism, master, sparkMemory);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
@@ -307,8 +290,7 @@ public class CompactionCommand implements CommandMarker {
|
||||
String message = "\n\n\t COMPACTION PLAN " + (valid ? "VALID" : "INVALID") + "\n\n";
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
res.stream().forEach(r -> {
|
||||
Comparable[] row = new Comparable[]{r.getOperation().getFileId(),
|
||||
r.getOperation().getBaseInstantTime(),
|
||||
Comparable[] row = new Comparable[] {r.getOperation().getFileId(), r.getOperation().getBaseInstantTime(),
|
||||
r.getOperation().getDataFilePath().isPresent() ? r.getOperation().getDataFilePath().get() : "",
|
||||
r.getOperation().getDeltaFilePaths().size(), r.isSuccess(),
|
||||
r.getException().isPresent() ? r.getException().get().getMessage() : ""};
|
||||
@@ -316,12 +298,8 @@ public class CompactionCommand implements CommandMarker {
|
||||
});
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Base Instant Time")
|
||||
.addTableHeaderField("Base Data File")
|
||||
.addTableHeaderField("Num Delta Files")
|
||||
.addTableHeaderField("Valid")
|
||||
TableHeader header = new TableHeader().addTableHeaderField("File Id").addTableHeaderField("Base Instant Time")
|
||||
.addTableHeaderField("Base Data File").addTableHeaderField("Num Delta Files").addTableHeaderField("Valid")
|
||||
.addTableHeaderField("Error");
|
||||
|
||||
output = message + HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit,
|
||||
@@ -349,8 +327,8 @@ public class CompactionCommand implements CommandMarker {
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
@@ -360,12 +338,12 @@ public class CompactionCommand implements CommandMarker {
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
String sparkPropertiesPath = Utils
|
||||
.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_PLAN.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory, Boolean.valueOf(skipV).toString(), Boolean.valueOf(dryRun).toString());
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_PLAN.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
compactionInstant, outputPathStr, parallelism, master, sparkMemory, Boolean.valueOf(skipV).toString(),
|
||||
Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
@@ -373,8 +351,8 @@ public class CompactionCommand implements CommandMarker {
|
||||
return "Failed to unschedule compaction for " + compactionInstant;
|
||||
}
|
||||
List<RenameOpResult> res = deSerializeOperationResult(outputPathStr, HoodieCLI.fs);
|
||||
output = getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly,
|
||||
"unschedule pending compaction");
|
||||
output =
|
||||
getRenamesToBePrinted(res, limit, sortByField, descending, headerOnly, "unschedule pending compaction");
|
||||
} finally {
|
||||
// Delete tmp file used to serialize result
|
||||
if (HoodieCLI.fs.exists(outputPath)) {
|
||||
@@ -407,12 +385,12 @@ public class CompactionCommand implements CommandMarker {
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
String sparkPropertiesPath = Utils
|
||||
.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_FILE.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), fileId, outputPathStr, "1", master,
|
||||
sparkMemory, Boolean.valueOf(skipV).toString(), Boolean.valueOf(dryRun).toString());
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_UNSCHEDULE_FILE.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
fileId, outputPathStr, "1", master, sparkMemory, Boolean.valueOf(skipV).toString(),
|
||||
Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
@@ -445,8 +423,8 @@ public class CompactionCommand implements CommandMarker {
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") boolean headerOnly)
|
||||
throws Exception {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
@@ -455,12 +433,11 @@ public class CompactionCommand implements CommandMarker {
|
||||
String output = "";
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
try {
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
String sparkPropertiesPath = Utils
|
||||
.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_REPAIR.toString(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), compactionInstant, outputPathStr, parallelism, master,
|
||||
sparkMemory, Boolean.valueOf(dryRun).toString());
|
||||
sparkLauncher.addAppArgs(SparkCommand.COMPACT_REPAIR.toString(), HoodieCLI.tableMetadata.getBasePath(),
|
||||
compactionInstant, outputPathStr, parallelism, master, sparkMemory, Boolean.valueOf(dryRun).toString());
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
@@ -481,41 +458,35 @@ public class CompactionCommand implements CommandMarker {
|
||||
}
|
||||
}
|
||||
|
||||
private String getRenamesToBePrinted(List<RenameOpResult> res, Integer limit,
|
||||
String sortByField, boolean descending, boolean headerOnly, String operation) {
|
||||
private String getRenamesToBePrinted(List<RenameOpResult> res, Integer limit, String sortByField, boolean descending,
|
||||
boolean headerOnly, String operation) {
|
||||
|
||||
Option<Boolean> result = Option.fromJavaOptional(
|
||||
res.stream().map(r -> r.isExecuted() && r.isSuccess()).reduce(Boolean::logicalAnd));
|
||||
Option<Boolean> result =
|
||||
Option.fromJavaOptional(res.stream().map(r -> r.isExecuted() && r.isSuccess()).reduce(Boolean::logicalAnd));
|
||||
if (result.isPresent()) {
|
||||
System.out.println("There were some file renames that needed to be done to " + operation);
|
||||
|
||||
if (result.get()) {
|
||||
System.out.println("All renames successfully completed to " + operation + " done !!");
|
||||
} else {
|
||||
System.out.println("Some renames failed. DataSet could be in inconsistent-state. "
|
||||
+ "Try running compaction repair");
|
||||
System.out
|
||||
.println("Some renames failed. DataSet could be in inconsistent-state. " + "Try running compaction repair");
|
||||
}
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
res.stream().forEach(r -> {
|
||||
Comparable[] row = new Comparable[] {
|
||||
r.getOperation().fileId, r.getOperation().srcPath, r.getOperation().destPath,
|
||||
r.isExecuted(), r.isSuccess(), r.getException().isPresent() ? r.getException().get().getMessage() : ""
|
||||
};
|
||||
Comparable[] row =
|
||||
new Comparable[] {r.getOperation().fileId, r.getOperation().srcPath, r.getOperation().destPath,
|
||||
r.isExecuted(), r.isSuccess(), r.getException().isPresent() ? r.getException().get().getMessage() : ""};
|
||||
rows.add(row);
|
||||
});
|
||||
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("File Id")
|
||||
.addTableHeaderField("Source File Path")
|
||||
.addTableHeaderField("Destination File Path")
|
||||
.addTableHeaderField("Rename Executed?")
|
||||
.addTableHeaderField("Rename Succeeded?")
|
||||
.addTableHeaderField("Error");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("File Id").addTableHeaderField("Source File Path")
|
||||
.addTableHeaderField("Destination File Path").addTableHeaderField("Rename Executed?")
|
||||
.addTableHeaderField("Rename Succeeded?").addTableHeaderField("Error");
|
||||
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending,
|
||||
limit, headerOnly, rows);
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
} else {
|
||||
return "No File renames needed to " + operation + ". Operation successful.";
|
||||
}
|
||||
|
||||
@@ -52,13 +52,12 @@ public class DatasetsCommand implements CommandMarker {
|
||||
@CliOption(key = {"maxCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "300000",
|
||||
help = "Max wait time for eventual consistency") final Integer maxConsistencyIntervalMs,
|
||||
@CliOption(key = {"maxCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "7",
|
||||
help = "Max checks for eventual consistency") final Integer maxConsistencyChecks) throws IOException {
|
||||
HoodieCLI.setConsistencyGuardConfig(
|
||||
ConsistencyGuardConfig.newBuilder()
|
||||
.withConsistencyCheckEnabled(eventuallyConsistent)
|
||||
help = "Max checks for eventual consistency") final Integer maxConsistencyChecks)
|
||||
throws IOException {
|
||||
HoodieCLI
|
||||
.setConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(eventuallyConsistent)
|
||||
.withInitialConsistencyCheckIntervalMs(initialConsistencyIntervalMs)
|
||||
.withMaxConsistencyCheckIntervalMs(maxConsistencyIntervalMs)
|
||||
.withMaxConsistencyChecks(maxConsistencyChecks)
|
||||
.withMaxConsistencyCheckIntervalMs(maxConsistencyIntervalMs).withMaxConsistencyChecks(maxConsistencyChecks)
|
||||
.build());
|
||||
HoodieCLI.initConf();
|
||||
HoodieCLI.connectTo(path);
|
||||
@@ -70,8 +69,8 @@ public class DatasetsCommand implements CommandMarker {
|
||||
/**
|
||||
* Create a Hoodie Table if it does not exist
|
||||
*
|
||||
* @param path Base Path
|
||||
* @param name Hoodie Table Name
|
||||
* @param path Base Path
|
||||
* @param name Hoodie Table Name
|
||||
* @param tableTypeStr Hoodie Table Type
|
||||
* @param payloadClass Payload Class
|
||||
*/
|
||||
@@ -82,7 +81,8 @@ public class DatasetsCommand implements CommandMarker {
|
||||
@CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE",
|
||||
help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr,
|
||||
@CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.HoodieAvroPayload",
|
||||
help = "Payload Class") final String payloadClass) throws IOException {
|
||||
help = "Payload Class") final String payloadClass)
|
||||
throws IOException {
|
||||
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
@@ -117,15 +117,13 @@ public class DatasetsCommand implements CommandMarker {
|
||||
*/
|
||||
@CliCommand(value = "desc", help = "Describle Hoodie Table properties")
|
||||
public String descTable() {
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Property")
|
||||
.addTableHeaderField("Value");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Property").addTableHeaderField("Value");
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
rows.add(new Comparable[]{"basePath", HoodieCLI.tableMetadata.getBasePath()});
|
||||
rows.add(new Comparable[]{"metaPath", HoodieCLI.tableMetadata.getMetaPath()});
|
||||
rows.add(new Comparable[]{"fileSystem", HoodieCLI.tableMetadata.getFs().getScheme()});
|
||||
rows.add(new Comparable[] {"basePath", HoodieCLI.tableMetadata.getBasePath()});
|
||||
rows.add(new Comparable[] {"metaPath", HoodieCLI.tableMetadata.getMetaPath()});
|
||||
rows.add(new Comparable[] {"fileSystem", HoodieCLI.tableMetadata.getFs().getScheme()});
|
||||
HoodieCLI.tableMetadata.getTableConfig().getProps().entrySet().forEach(e -> {
|
||||
rows.add(new Comparable[]{e.getKey(), e.getValue()});
|
||||
rows.add(new Comparable[] {e.getKey(), e.getValue()});
|
||||
});
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows);
|
||||
}
|
||||
|
||||
@@ -52,24 +52,23 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "show fsview all", help = "Show entire file-system view")
|
||||
public String showAllFileSlices(
|
||||
@CliOption(key = {"pathRegex"},
|
||||
help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") String globRegex,
|
||||
@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 = {"maxInstant"}, help = "File-Slices upto this instant are displayed",
|
||||
unspecifiedDefaultValue = "") String maxInstant,
|
||||
@CliOption(key = {
|
||||
"includeMax"}, help = "Include Max Instant", unspecifiedDefaultValue = "false") boolean includeMaxInstant,
|
||||
@CliOption(key = {
|
||||
"includeInflight"}, help = "Include Inflight Instants", unspecifiedDefaultValue = "false")
|
||||
boolean includeInflight,
|
||||
@CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants", unspecifiedDefaultValue = "false")
|
||||
boolean excludeCompaction,
|
||||
@CliOption(key = {"includeMax"}, help = "Include Max Instant",
|
||||
unspecifiedDefaultValue = "false") boolean includeMaxInstant,
|
||||
@CliOption(key = {"includeInflight"}, help = "Include Inflight Instants",
|
||||
unspecifiedDefaultValue = "false") boolean includeInflight,
|
||||
@CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants",
|
||||
unspecifiedDefaultValue = "false") boolean excludeCompaction,
|
||||
@CliOption(key = {"limit"}, help = "Limit rows to be displayed", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieTableFileSystemView fsView = buildFileSystemView(globRegex, maxInstant, readOptimizedOnly, includeMaxInstant,
|
||||
@@ -97,15 +96,10 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
fieldNameToConverterMap.put("Total Delta File Size", converterFunction);
|
||||
fieldNameToConverterMap.put("Data-File Size", converterFunction);
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition")
|
||||
.addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant")
|
||||
.addTableHeaderField("Data-File")
|
||||
.addTableHeaderField("Data-File Size");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition").addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant").addTableHeaderField("Data-File").addTableHeaderField("Data-File Size");
|
||||
if (!readOptimizedOnly) {
|
||||
header = header.addTableHeaderField("Num Delta Files")
|
||||
.addTableHeaderField("Total Delta File Size")
|
||||
header = header.addTableHeaderField("Num Delta Files").addTableHeaderField("Total Delta File Size")
|
||||
.addTableHeaderField("Delta Files");
|
||||
}
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
@@ -113,25 +107,24 @@ 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 = {"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 = {"maxInstant"}, help = "File-Slices upto this instant are displayed",
|
||||
unspecifiedDefaultValue = "") String maxInstant,
|
||||
@CliOption(key = {"merge"}, help = "Merge File Slices due to pending compaction",
|
||||
unspecifiedDefaultValue = "true") final boolean merge,
|
||||
@CliOption(key = {"includeMax"}, help = "Include Max Instant", unspecifiedDefaultValue = "false")
|
||||
boolean includeMaxInstant,
|
||||
@CliOption(key = {"includeInflight"}, help = "Include Inflight Instants", unspecifiedDefaultValue = "false")
|
||||
boolean includeInflight,
|
||||
@CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants", unspecifiedDefaultValue = "false")
|
||||
boolean excludeCompaction,
|
||||
@CliOption(key = {"includeMax"}, help = "Include Max Instant",
|
||||
unspecifiedDefaultValue = "false") boolean includeMaxInstant,
|
||||
@CliOption(key = {"includeInflight"}, help = "Include Inflight Instants",
|
||||
unspecifiedDefaultValue = "false") boolean includeInflight,
|
||||
@CliOption(key = {"excludeCompaction"}, help = "Exclude compaction Instants",
|
||||
unspecifiedDefaultValue = "false") boolean excludeCompaction,
|
||||
@CliOption(key = {"limit"}, help = "Limit rows to be displayed", unspecifiedDefaultValue = "-1") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
HoodieTableFileSystemView fsView = buildFileSystemView(partition, maxInstant, readOptimizedOnly, includeMaxInstant,
|
||||
@@ -163,28 +156,25 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
if (!readOptimizedOnly) {
|
||||
row[idx++] = fs.getLogFiles().count();
|
||||
row[idx++] = fs.getLogFiles().mapToLong(lf -> lf.getFileSize()).sum();
|
||||
long logFilesScheduledForCompactionTotalSize = fs.getLogFiles()
|
||||
.filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.mapToLong(lf -> lf.getFileSize()).sum();
|
||||
long logFilesScheduledForCompactionTotalSize =
|
||||
fs.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.mapToLong(lf -> lf.getFileSize()).sum();
|
||||
row[idx++] = logFilesScheduledForCompactionTotalSize;
|
||||
|
||||
long logFilesUnscheduledTotalSize = fs.getLogFiles()
|
||||
.filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.mapToLong(lf -> lf.getFileSize()).sum();
|
||||
long logFilesUnscheduledTotalSize =
|
||||
fs.getLogFiles().filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.mapToLong(lf -> lf.getFileSize()).sum();
|
||||
row[idx++] = logFilesUnscheduledTotalSize;
|
||||
|
||||
double logSelectedForCompactionToBaseRatio =
|
||||
dataFileSize > 0 ? logFilesScheduledForCompactionTotalSize / (dataFileSize * 1.0) : -1;
|
||||
row[idx++] = logSelectedForCompactionToBaseRatio;
|
||||
double logUnscheduledToBaseRatio =
|
||||
dataFileSize > 0 ? logFilesUnscheduledTotalSize / (dataFileSize * 1.0) : -1;
|
||||
double logUnscheduledToBaseRatio = dataFileSize > 0 ? logFilesUnscheduledTotalSize / (dataFileSize * 1.0) : -1;
|
||||
row[idx++] = logUnscheduledToBaseRatio;
|
||||
|
||||
row[idx++] = fs.getLogFiles()
|
||||
.filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
row[idx++] = fs.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
row[idx++] = fs.getLogFiles()
|
||||
.filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
row[idx++] = fs.getLogFiles().filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
}
|
||||
rows.add(row);
|
||||
@@ -200,16 +190,11 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
fieldNameToConverterMap.put("Delta Size - compaction unscheduled", converterFunction);
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Partition")
|
||||
.addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant")
|
||||
.addTableHeaderField("Data-File")
|
||||
.addTableHeaderField("Data-File Size");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition").addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant").addTableHeaderField("Data-File").addTableHeaderField("Data-File Size");
|
||||
|
||||
if (!readOptimizedOnly) {
|
||||
header = header.addTableHeaderField("Num Delta Files")
|
||||
.addTableHeaderField("Total Delta Size")
|
||||
header = header.addTableHeaderField("Num Delta Files").addTableHeaderField("Total Delta Size")
|
||||
.addTableHeaderField("Delta Size - compaction scheduled")
|
||||
.addTableHeaderField("Delta Size - compaction unscheduled")
|
||||
.addTableHeaderField("Delta To Base Ratio - compaction scheduled")
|
||||
@@ -222,19 +207,20 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
|
||||
/**
|
||||
* Build File System View
|
||||
*
|
||||
* @param globRegex Path Regex
|
||||
* @param maxInstant Max Instants to be used for displaying file-instants
|
||||
* @param maxInstant Max Instants to be used for displaying file-instants
|
||||
* @param readOptimizedOnly Include only read optimized view
|
||||
* @param includeMaxInstant Include Max instant
|
||||
* @param includeInflight Include inflight instants
|
||||
* @param includeInflight Include inflight instants
|
||||
* @param excludeCompaction Exclude Compaction instants
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private HoodieTableFileSystemView buildFileSystemView(String globRegex, String maxInstant, boolean readOptimizedOnly,
|
||||
boolean includeMaxInstant, boolean includeInflight, boolean excludeCompaction) throws IOException {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieCLI.tableMetadata.getHadoopConf(),
|
||||
HoodieCLI.tableMetadata.getBasePath(), true);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(HoodieCLI.tableMetadata.getHadoopConf(), HoodieCLI.tableMetadata.getBasePath(), true);
|
||||
FileSystem fs = HoodieCLI.fs;
|
||||
String globPath = String.format("%s/%s/*", HoodieCLI.tableMetadata.getBasePath(), globRegex);
|
||||
FileStatus[] statuses = fs.globStatus(new Path(globPath));
|
||||
|
||||
@@ -43,17 +43,17 @@ public class HDFSParquetImportCommand implements CommandMarker {
|
||||
@CliOption(key = "upsert", mandatory = false, unspecifiedDefaultValue = "false",
|
||||
help = "Uses upsert API instead of the default insert API of WriteClient") boolean useUpsert,
|
||||
@CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") final String srcPath,
|
||||
@CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset") final String
|
||||
targetPath,
|
||||
@CliOption(key = "targetPath", mandatory = true,
|
||||
help = "Base path for the target hoodie dataset") final String targetPath,
|
||||
@CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName,
|
||||
@CliOption(key = "tableType", mandatory = true, help = "Table type") final String tableType,
|
||||
@CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField,
|
||||
@CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name") final String
|
||||
partitionPathField,
|
||||
@CliOption(key = {
|
||||
"parallelism"}, mandatory = true, help = "Parallelism for hoodie insert") final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file") final String
|
||||
schemaFilePath,
|
||||
@CliOption(key = "partitionPathField", mandatory = true,
|
||||
help = "Partition path field name") final String partitionPathField,
|
||||
@CliOption(key = {"parallelism"}, mandatory = true,
|
||||
help = "Parallelism for hoodie insert") final String parallelism,
|
||||
@CliOption(key = "schemaFilePath", mandatory = true,
|
||||
help = "Path for Avro schema file") final String schemaFilePath,
|
||||
@CliOption(key = "format", mandatory = true, help = "Format for the input data") final String format,
|
||||
@CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory") final String sparkMemory,
|
||||
@CliOption(key = "retry", mandatory = true, help = "Number of retries") final String retry) throws Exception {
|
||||
@@ -62,8 +62,8 @@ public class HDFSParquetImportCommand implements CommandMarker {
|
||||
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
String sparkPropertiesPath = Utils.getDefaultPropertiesFile(
|
||||
JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
|
||||
String sparkPropertiesPath =
|
||||
Utils.getDefaultPropertiesFile(JavaConverters.mapAsScalaMapConverter(System.getenv()).asScala());
|
||||
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
|
||||
@@ -72,8 +72,8 @@ public class HDFSParquetImportCommand implements CommandMarker {
|
||||
cmd = SparkCommand.UPSERT.toString();
|
||||
}
|
||||
|
||||
sparkLauncher.addAppArgs(cmd, srcPath, targetPath, tableName, tableType, rowKeyField,
|
||||
partitionPathField, parallelism, schemaFilePath, sparkMemory, retry);
|
||||
sparkLauncher.addAppArgs(cmd, srcPath, targetPath, tableName, tableType, rowKeyField, partitionPathField,
|
||||
parallelism, schemaFilePath, sparkMemory, retry);
|
||||
Process process = sparkLauncher.launch();
|
||||
InputStreamConsumer.captureOutput(process);
|
||||
int exitCode = process.waitFor();
|
||||
|
||||
@@ -69,30 +69,29 @@ public class HoodieLogFileCommand implements CommandMarker {
|
||||
|
||||
@CliCommand(value = "show logfile metadata", help = "Read commit metadata from log files")
|
||||
public String showLogFileCommits(
|
||||
@CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified path for the log file") final
|
||||
String logFilePathPattern,
|
||||
@CliOption(key = "logFilePathPattern", mandatory = true,
|
||||
help = "Fully qualified path for the log file") final String logFilePathPattern,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false")
|
||||
final boolean headerOnly) throws IOException {
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
FileSystem fs = HoodieCLI.tableMetadata.getFs();
|
||||
List<String> logFilePaths = Arrays.stream(fs.globStatus(new Path(logFilePathPattern)))
|
||||
.map(status -> status.getPath().toString()).collect(Collectors.toList());
|
||||
Map<String, List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType,
|
||||
String>>, Integer>>>
|
||||
commitCountAndMetadata = Maps.newHashMap();
|
||||
Map<String, List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer>>> commitCountAndMetadata =
|
||||
Maps.newHashMap();
|
||||
int totalEntries = 0;
|
||||
int numCorruptBlocks = 0;
|
||||
int dummyInstantTimeCount = 0;
|
||||
|
||||
for (String logFilePath : logFilePaths) {
|
||||
FileStatus[] fsStatus = fs.listStatus(new Path(logFilePath));
|
||||
Schema writerSchema = new AvroSchemaConverter().convert(
|
||||
SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath)));
|
||||
Reader reader = HoodieLogFormat
|
||||
.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema);
|
||||
Schema writerSchema = new AvroSchemaConverter()
|
||||
.convert(SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFilePath)));
|
||||
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema);
|
||||
|
||||
// read the avro blocks
|
||||
while (reader.hasNext()) {
|
||||
@@ -126,8 +125,8 @@ public class HoodieLogFileCommand implements CommandMarker {
|
||||
new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount));
|
||||
totalEntries++;
|
||||
} else {
|
||||
List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>,
|
||||
Integer>> list = new ArrayList<>();
|
||||
List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer>> list =
|
||||
new ArrayList<>();
|
||||
list.add(
|
||||
new Tuple3<>(n.getBlockType(), new Tuple2<>(n.getLogBlockHeader(), n.getLogBlockFooter()), recordCount));
|
||||
commitCountAndMetadata.put(instantTime, list);
|
||||
@@ -139,12 +138,11 @@ public class HoodieLogFileCommand implements CommandMarker {
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
int i = 0;
|
||||
ObjectMapper objectMapper = new ObjectMapper();
|
||||
for (Map.Entry<String, List<Tuple3<HoodieLogBlockType,
|
||||
Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer>>> entry
|
||||
: commitCountAndMetadata.entrySet()) {
|
||||
for (Map.Entry<String, List<Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer>>> entry : commitCountAndMetadata
|
||||
.entrySet()) {
|
||||
String instantTime = entry.getKey().toString();
|
||||
for (Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>,
|
||||
Map<HeaderMetadataType, String>>, Integer> tuple3 : entry.getValue()) {
|
||||
for (Tuple3<HoodieLogBlockType, Tuple2<Map<HeaderMetadataType, String>, Map<HeaderMetadataType, String>>, Integer> tuple3 : entry
|
||||
.getValue()) {
|
||||
Comparable[] output = new Comparable[5];
|
||||
output[0] = instantTime;
|
||||
output[1] = tuple3._3();
|
||||
@@ -156,21 +154,18 @@ public class HoodieLogFileCommand implements CommandMarker {
|
||||
}
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("InstantTime")
|
||||
.addTableHeaderField("RecordCount")
|
||||
.addTableHeaderField("BlockType")
|
||||
.addTableHeaderField("HeaderMetadata")
|
||||
.addTableHeaderField("FooterMetadata");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("InstantTime").addTableHeaderField("RecordCount")
|
||||
.addTableHeaderField("BlockType").addTableHeaderField("HeaderMetadata").addTableHeaderField("FooterMetadata");
|
||||
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@CliCommand(value = "show logfile records", help = "Read records from log files")
|
||||
public String showLogFileRecords(@CliOption(key = {
|
||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") final Integer limit,
|
||||
@CliOption(key = "logFilePathPattern", mandatory = true, help = "Fully qualified paths for the log files")
|
||||
final String logFilePathPattern,
|
||||
public String showLogFileRecords(
|
||||
@CliOption(key = {"limit"}, mandatory = false, help = "Limit commits",
|
||||
unspecifiedDefaultValue = "10") final Integer limit,
|
||||
@CliOption(key = "logFilePathPattern", mandatory = true,
|
||||
help = "Fully qualified paths for the log files") final String logFilePathPattern,
|
||||
@CliOption(key = "mergeRecords", mandatory = false, help = "If the records in the log files should be merged",
|
||||
unspecifiedDefaultValue = "false") final Boolean shouldMerge)
|
||||
throws IOException {
|
||||
@@ -184,22 +179,21 @@ public class HoodieLogFileCommand implements CommandMarker {
|
||||
// TODO : readerSchema can change across blocks/log files, fix this inside Scanner
|
||||
AvroSchemaConverter converter = new AvroSchemaConverter();
|
||||
// get schema from last log file
|
||||
Schema readerSchema = converter.convert(
|
||||
SchemaUtil.readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1))));
|
||||
Schema readerSchema =
|
||||
converter.convert(SchemaUtil.readSchemaFromLogFile(fs, new Path(logFilePaths.get(logFilePaths.size() - 1))));
|
||||
|
||||
List<IndexedRecord> allRecords = new ArrayList<>();
|
||||
|
||||
if (shouldMerge) {
|
||||
System.out.println("===========================> MERGING RECORDS <===================");
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
|
||||
HoodieCLI.tableMetadata.getBasePath(), logFilePaths, readerSchema,
|
||||
HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get()
|
||||
.getTimestamp(),
|
||||
Long.valueOf(HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES),
|
||||
Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED),
|
||||
Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED),
|
||||
Integer.valueOf(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
|
||||
HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH);
|
||||
HoodieMergedLogRecordScanner scanner =
|
||||
new HoodieMergedLogRecordScanner(fs, HoodieCLI.tableMetadata.getBasePath(), logFilePaths, readerSchema,
|
||||
HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get().getTimestamp(),
|
||||
Long.valueOf(HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES),
|
||||
Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED),
|
||||
Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED),
|
||||
Integer.valueOf(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
|
||||
HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH);
|
||||
for (HoodieRecord<? extends HoodieRecordPayload> hoodieRecord : scanner) {
|
||||
Option<IndexedRecord> record = hoodieRecord.getData().getInsertValue(readerSchema);
|
||||
if (allRecords.size() >= limit) {
|
||||
@@ -209,10 +203,10 @@ public class HoodieLogFileCommand implements CommandMarker {
|
||||
}
|
||||
} else {
|
||||
for (String logFile : logFilePaths) {
|
||||
Schema writerSchema = new AvroSchemaConverter().convert(
|
||||
SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFile)));
|
||||
HoodieLogFormat.Reader reader = HoodieLogFormat
|
||||
.newReader(fs, new HoodieLogFile(new Path(logFile)), writerSchema);
|
||||
Schema writerSchema = new AvroSchemaConverter()
|
||||
.convert(SchemaUtil.readSchemaFromLogFile(HoodieCLI.tableMetadata.getFs(), new Path(logFile)));
|
||||
HoodieLogFormat.Reader reader =
|
||||
HoodieLogFormat.newReader(fs, new HoodieLogFile(new Path(logFile)), writerSchema);
|
||||
// read the avro blocks
|
||||
while (reader.hasNext()) {
|
||||
HoodieLogBlock n = reader.next();
|
||||
|
||||
@@ -44,19 +44,16 @@ public class HoodieSyncCommand implements CommandMarker {
|
||||
public String validateSync(
|
||||
@CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode") final String mode,
|
||||
@CliOption(key = {"sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database") final String srcDb,
|
||||
@CliOption(key = {
|
||||
"targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database") final String tgtDb,
|
||||
@CliOption(key = {
|
||||
"partitionCount"}, unspecifiedDefaultValue = "5", help = "total number of recent partitions to validate")
|
||||
final int partitionCount,
|
||||
@CliOption(key = {
|
||||
"hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to") final String hiveServerUrl,
|
||||
@CliOption(key = {
|
||||
"hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to") final
|
||||
String hiveUser,
|
||||
@CliOption(key = {
|
||||
"hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") final
|
||||
String hivePass)
|
||||
@CliOption(key = {"targetDb"}, unspecifiedDefaultValue = "dwh_hoodie",
|
||||
help = "target database") final String tgtDb,
|
||||
@CliOption(key = {"partitionCount"}, unspecifiedDefaultValue = "5",
|
||||
help = "total number of recent partitions to validate") final int partitionCount,
|
||||
@CliOption(key = {"hiveServerUrl"}, mandatory = true,
|
||||
help = "hiveServerURL to connect to") final String hiveServerUrl,
|
||||
@CliOption(key = {"hiveUser"}, mandatory = false, unspecifiedDefaultValue = "",
|
||||
help = "hive username to connect to") final String hiveUser,
|
||||
@CliOption(key = {"hivePass"}, mandatory = true, unspecifiedDefaultValue = "",
|
||||
help = "hive password to connect to") final String hivePass)
|
||||
throws Exception {
|
||||
HoodieTableMetaClient target = HoodieCLI.syncTableMetadata;
|
||||
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline();
|
||||
@@ -77,8 +74,8 @@ public class HoodieSyncCommand implements CommandMarker {
|
||||
String sourceLatestCommit =
|
||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp();
|
||||
|
||||
if (sourceLatestCommit != null && HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit,
|
||||
HoodieTimeline.GREATER)) {
|
||||
if (sourceLatestCommit != null
|
||||
&& HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
||||
// source is behind the target
|
||||
List<HoodieInstant> commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
|
||||
.getInstants().collect(Collectors.toList());
|
||||
@@ -89,8 +86,8 @@ public class HoodieSyncCommand implements CommandMarker {
|
||||
long newInserts = CommitUtil.countNewRecords(target,
|
||||
commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()));
|
||||
return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count("
|
||||
+ source.getTableConfig().getTableName()
|
||||
+ ") == " + (targetCount - sourceCount) + ". Catch up count is " + newInserts;
|
||||
+ source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount) + ". Catch up count is "
|
||||
+ newInserts;
|
||||
}
|
||||
} else {
|
||||
List<HoodieInstant> commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
|
||||
@@ -102,8 +99,8 @@ public class HoodieSyncCommand implements CommandMarker {
|
||||
long newInserts = CommitUtil.countNewRecords(source,
|
||||
commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()));
|
||||
return "Count difference now is (count(" + source.getTableConfig().getTableName() + ") - count("
|
||||
+ target.getTableConfig().getTableName()
|
||||
+ ") == " + (sourceCount - targetCount) + ". Catch up count is " + newInserts;
|
||||
+ target.getTableConfig().getTableName() + ") == " + (sourceCount - targetCount) + ". Catch up count is "
|
||||
+ newInserts;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -47,16 +47,15 @@ public class RepairsCommand implements CommandMarker {
|
||||
return HoodieCLI.tableMetadata != null;
|
||||
}
|
||||
|
||||
@CliCommand(value = "repair deduplicate", help = "De-duplicate a partition path contains duplicates & produce "
|
||||
+ "repaired files to replace with")
|
||||
public String deduplicate(@CliOption(key = {
|
||||
"duplicatedPartitionPath"}, help = "Partition Path containing the duplicates", mandatory = true) final String
|
||||
duplicatedPartitionPath,
|
||||
@CliOption(key = {
|
||||
"repairedOutputPath"}, help = "Location to place the repaired files", mandatory = true) final String
|
||||
repairedOutputPath,
|
||||
@CliOption(key = {
|
||||
"sparkProperties"}, help = "Spark Properites File Path", mandatory = true) final String sparkPropertiesPath)
|
||||
@CliCommand(value = "repair deduplicate",
|
||||
help = "De-duplicate a partition path contains duplicates & produce " + "repaired files to replace with")
|
||||
public String deduplicate(
|
||||
@CliOption(key = {"duplicatedPartitionPath"}, help = "Partition Path containing the duplicates",
|
||||
mandatory = true) final String duplicatedPartitionPath,
|
||||
@CliOption(key = {"repairedOutputPath"}, help = "Location to place the repaired files",
|
||||
mandatory = true) final String repairedOutputPath,
|
||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path",
|
||||
mandatory = true) final String sparkPropertiesPath)
|
||||
throws Exception {
|
||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||
sparkLauncher.addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath, repairedOutputPath,
|
||||
@@ -73,14 +72,15 @@ public class RepairsCommand implements CommandMarker {
|
||||
|
||||
|
||||
@CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present")
|
||||
public String addPartitionMeta(@CliOption(key = {
|
||||
"dryrun"}, help = "Should we actually add or just print what would be done", unspecifiedDefaultValue = "true")
|
||||
final boolean dryRun) throws IOException {
|
||||
public String addPartitionMeta(
|
||||
@CliOption(key = {"dryrun"}, help = "Should we actually add or just print what would be done",
|
||||
unspecifiedDefaultValue = "true") final boolean dryRun)
|
||||
throws IOException {
|
||||
|
||||
String latestCommit = HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get()
|
||||
.getTimestamp();
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionFoldersThreeLevelsDown(HoodieCLI.fs,
|
||||
HoodieCLI.tableMetadata.getBasePath());
|
||||
String latestCommit =
|
||||
HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get().getTimestamp();
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionFoldersThreeLevelsDown(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
||||
Path basePath = new Path(HoodieCLI.tableMetadata.getBasePath());
|
||||
String[][] rows = new String[partitionPaths.size() + 1][];
|
||||
|
||||
@@ -94,8 +94,8 @@ public class RepairsCommand implements CommandMarker {
|
||||
if (!HoodiePartitionMetadata.hasPartitionMetadata(HoodieCLI.fs, partitionPath)) {
|
||||
row[1] = "No";
|
||||
if (!dryRun) {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath,
|
||||
partitionPath);
|
||||
HoodiePartitionMetadata partitionMetadata =
|
||||
new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partitionPath);
|
||||
partitionMetadata.trySave(0);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -50,8 +50,8 @@ public class RollbacksCommand implements CommandMarker {
|
||||
@CliOption(key = {"limit"}, help = "Limit #rows to be displayed", unspecifiedDefaultValue = "10") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = new RollbackTimeline(HoodieCLI.tableMetadata);
|
||||
HoodieTimeline rollback = activeTimeline.getRollbackTimeline().filterCompletedInstants();
|
||||
@@ -59,8 +59,8 @@ public class RollbacksCommand implements CommandMarker {
|
||||
final List<Comparable[]> rows = new ArrayList<>();
|
||||
rollback.getInstants().forEach(instant -> {
|
||||
try {
|
||||
HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata(
|
||||
activeTimeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class);
|
||||
HoodieRollbackMetadata metadata = AvroUtils
|
||||
.deserializeAvroMetadata(activeTimeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class);
|
||||
metadata.getCommitsRollback().forEach(c -> {
|
||||
Comparable[] row = new Comparable[5];
|
||||
row[0] = metadata.getStartRollbackTime();
|
||||
@@ -74,11 +74,8 @@ public class RollbacksCommand implements CommandMarker {
|
||||
e.printStackTrace();
|
||||
}
|
||||
});
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Instant")
|
||||
.addTableHeaderField("Rolledback Instant")
|
||||
.addTableHeaderField("Total Files Deleted")
|
||||
.addTableHeaderField("Time taken in millis")
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Instant").addTableHeaderField("Rolledback Instant")
|
||||
.addTableHeaderField("Total Files Deleted").addTableHeaderField("Time taken in millis")
|
||||
.addTableHeaderField("Total Partitions");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
@@ -89,17 +86,18 @@ public class RollbacksCommand implements CommandMarker {
|
||||
@CliOption(key = {"limit"}, help = "Limit #rows to be displayed", unspecifiedDefaultValue = "10") Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {
|
||||
"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
HoodieActiveTimeline activeTimeline = new RollbackTimeline(HoodieCLI.tableMetadata);
|
||||
final List<Comparable[]> rows = new ArrayList<>();
|
||||
HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata(
|
||||
activeTimeline.getInstantDetails(new HoodieInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant))
|
||||
.get(), HoodieRollbackMetadata.class);
|
||||
activeTimeline.getInstantDetails(new HoodieInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant)).get(),
|
||||
HoodieRollbackMetadata.class);
|
||||
metadata.getPartitionMetadata().entrySet().forEach(e -> {
|
||||
Stream.concat(e.getValue().getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)),
|
||||
e.getValue().getFailedDeleteFiles().stream().map(f -> Pair.of(f, false)))
|
||||
Stream
|
||||
.concat(e.getValue().getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)),
|
||||
e.getValue().getFailedDeleteFiles().stream().map(f -> Pair.of(f, false)))
|
||||
.forEach(fileWithDeleteStatus -> {
|
||||
Comparable[] row = new Comparable[5];
|
||||
row[0] = metadata.getStartRollbackTime();
|
||||
@@ -111,12 +109,8 @@ public class RollbacksCommand implements CommandMarker {
|
||||
});
|
||||
});
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("Instant")
|
||||
.addTableHeaderField("Rolledback Instants")
|
||||
.addTableHeaderField("Partition")
|
||||
.addTableHeaderField("Deleted File")
|
||||
.addTableHeaderField("Succeeded");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Instant").addTableHeaderField("Rolledback Instants")
|
||||
.addTableHeaderField("Partition").addTableHeaderField("Deleted File").addTableHeaderField("Succeeded");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
|
||||
@@ -62,8 +62,8 @@ public class SavepointsCommand implements CommandMarker {
|
||||
|
||||
@CliAvailabilityIndicator({"savepoint rollback"})
|
||||
public boolean isRollbackToSavepointAvailable() {
|
||||
return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline().getSavePointTimeline()
|
||||
.filterCompletedInstants().empty();
|
||||
return HoodieCLI.tableMetadata != null
|
||||
&& !HoodieCLI.tableMetadata.getActiveTimeline().getSavePointTimeline().filterCompletedInstants().empty();
|
||||
}
|
||||
|
||||
@CliCommand(value = "savepoints show", help = "Show the savepoints")
|
||||
@@ -137,8 +137,8 @@ public class SavepointsCommand implements CommandMarker {
|
||||
}
|
||||
|
||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
return new HoodieWriteClient(jsc, config, false);
|
||||
}
|
||||
|
||||
|
||||
@@ -43,8 +43,7 @@ public class SparkMain {
|
||||
* Commands
|
||||
*/
|
||||
enum SparkCommand {
|
||||
ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN,
|
||||
COMPACT_UNSCHEDULE_PLAN, COMPACT_UNSCHEDULE_FILE, COMPACT_VALIDATE, COMPACT_REPAIR
|
||||
ROLLBACK, DEDUPLICATE, ROLLBACK_TO_SAVEPOINT, SAVEPOINT, IMPORT, UPSERT, COMPACT_SCHEDULE, COMPACT_RUN, COMPACT_UNSCHEDULE_PLAN, COMPACT_UNSCHEDULE_FILE, COMPACT_VALIDATE, COMPACT_REPAIR
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
@@ -76,13 +75,12 @@ public class SparkMain {
|
||||
break;
|
||||
case COMPACT_RUN:
|
||||
assert (args.length == 8);
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]),
|
||||
args[5], args[6], Integer.parseInt(args[7]), false);
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
|
||||
Integer.parseInt(args[7]), false);
|
||||
break;
|
||||
case COMPACT_SCHEDULE:
|
||||
assert (args.length == 5);
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], 1,
|
||||
"", args[4], 0, true);
|
||||
returnCode = compact(jsc, args[1], args[2], args[3], 1, "", args[4], 0, true);
|
||||
break;
|
||||
case COMPACT_VALIDATE:
|
||||
assert (args.length == 7);
|
||||
@@ -113,8 +111,7 @@ public class SparkMain {
|
||||
System.exit(returnCode);
|
||||
}
|
||||
|
||||
private static int dataLoad(JavaSparkContext jsc, String command,
|
||||
String srcPath, String targetPath, String tableName,
|
||||
private static int dataLoad(JavaSparkContext jsc, String command, String srcPath, String targetPath, String tableName,
|
||||
String tableType, String rowKey, String partitionKey, int parallelism, String schemaFile, String sparkMaster,
|
||||
String sparkMemory, int retry) throws Exception {
|
||||
Config cfg = new Config();
|
||||
@@ -180,9 +177,9 @@ public class SparkMain {
|
||||
new HoodieCompactionAdminTool(cfg).run(jsc);
|
||||
}
|
||||
|
||||
private static void doCompactUnscheduleFile(JavaSparkContext jsc, String basePath, String fileId,
|
||||
String outputPath, int parallelism, String sparkMaster, String sparkMemory, boolean skipValidation,
|
||||
boolean dryRun) throws Exception {
|
||||
private static void doCompactUnscheduleFile(JavaSparkContext jsc, String basePath, String fileId, String outputPath,
|
||||
int parallelism, String sparkMaster, String sparkMemory, boolean skipValidation, boolean dryRun)
|
||||
throws Exception {
|
||||
HoodieCompactionAdminTool.Config cfg = new HoodieCompactionAdminTool.Config();
|
||||
cfg.basePath = basePath;
|
||||
cfg.operation = Operation.UNSCHEDULE_FILE;
|
||||
@@ -244,8 +241,8 @@ public class SparkMain {
|
||||
}
|
||||
|
||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
return new HoodieWriteClient(jsc, config);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -63,8 +63,9 @@ public class StatsCommand implements CommandMarker {
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false")
|
||||
final boolean headerOnly) throws IOException {
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
long totalRecordsUpserted = 0;
|
||||
long totalRecordsWritten = 0;
|
||||
@@ -82,7 +83,7 @@ public class StatsCommand implements CommandMarker {
|
||||
if (commit.fetchTotalUpdateRecordsWritten() > 0) {
|
||||
waf = df.format((float) commit.fetchTotalRecordsWritten() / commit.fetchTotalUpdateRecordsWritten());
|
||||
}
|
||||
rows.add(new Comparable[]{commitTime.getTimestamp(), commit.fetchTotalUpdateRecordsWritten(),
|
||||
rows.add(new Comparable[] {commitTime.getTimestamp(), commit.fetchTotalUpdateRecordsWritten(),
|
||||
commit.fetchTotalRecordsWritten(), waf});
|
||||
totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten();
|
||||
totalRecordsWritten += commit.fetchTotalRecordsWritten();
|
||||
@@ -91,33 +92,28 @@ public class StatsCommand implements CommandMarker {
|
||||
if (totalRecordsUpserted > 0) {
|
||||
waf = df.format((float) totalRecordsWritten / totalRecordsUpserted);
|
||||
}
|
||||
rows.add(new Comparable[]{"Total", totalRecordsUpserted, totalRecordsWritten, waf});
|
||||
rows.add(new Comparable[] {"Total", totalRecordsUpserted, totalRecordsWritten, waf});
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("CommitTime")
|
||||
.addTableHeaderField("Total Upserted")
|
||||
.addTableHeaderField("Total Written")
|
||||
.addTableHeaderField("Write Amplifiation Factor");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("CommitTime").addTableHeaderField("Total Upserted")
|
||||
.addTableHeaderField("Total Written").addTableHeaderField("Write Amplifiation Factor");
|
||||
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
private Comparable[] printFileSizeHistogram(String commitTime, Snapshot s) {
|
||||
return new Comparable[]{commitTime, s.getMin(),
|
||||
s.getValue(0.1), s.getMedian(),
|
||||
s.getMean(), s.get95thPercentile(),
|
||||
s.getMax(), s.size(),
|
||||
s.getStdDev()};
|
||||
return new Comparable[] {commitTime, s.getMin(), s.getValue(0.1), s.getMedian(), s.getMean(), s.get95thPercentile(),
|
||||
s.getMax(), s.size(), s.getStdDev()};
|
||||
}
|
||||
|
||||
@CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files")
|
||||
public String fileSizeStats(
|
||||
@CliOption(key = {"partitionPath"},
|
||||
help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") final String globRegex,
|
||||
@CliOption(key = {"partitionPath"}, help = "regex to select files, eg: 2016/08/02",
|
||||
unspecifiedDefaultValue = "*/*/*") final String globRegex,
|
||||
@CliOption(key = {"limit"}, help = "Limit commits", unspecifiedDefaultValue = "-1") final Integer limit,
|
||||
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
|
||||
@CliOption(key = {"desc"}, help = "Ordering", unspecifiedDefaultValue = "false") final boolean descending,
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only", unspecifiedDefaultValue = "false")
|
||||
final boolean headerOnly) throws IOException {
|
||||
@CliOption(key = {"headeronly"}, help = "Print Header Only",
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws IOException {
|
||||
|
||||
FileSystem fs = HoodieCLI.fs;
|
||||
String globPath = String.format("%s/%s/*", HoodieCLI.tableMetadata.getBasePath(), globRegex);
|
||||
@@ -145,8 +141,8 @@ public class StatsCommand implements CommandMarker {
|
||||
Snapshot s = globalHistogram.getSnapshot();
|
||||
rows.add(printFileSizeHistogram("ALL", s));
|
||||
|
||||
Function<Object, String> converterFunction = entry ->
|
||||
NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
Function<Object, String> converterFunction =
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.valueOf(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Min", converterFunction);
|
||||
fieldNameToConverterMap.put("10th", converterFunction);
|
||||
@@ -156,16 +152,9 @@ public class StatsCommand implements CommandMarker {
|
||||
fieldNameToConverterMap.put("Max", converterFunction);
|
||||
fieldNameToConverterMap.put("StdDev", converterFunction);
|
||||
|
||||
TableHeader header = new TableHeader()
|
||||
.addTableHeaderField("CommitTime")
|
||||
.addTableHeaderField("Min")
|
||||
.addTableHeaderField("10th")
|
||||
.addTableHeaderField("50th")
|
||||
.addTableHeaderField("avg")
|
||||
.addTableHeaderField("95th")
|
||||
.addTableHeaderField("Max")
|
||||
.addTableHeaderField("NumFiles")
|
||||
.addTableHeaderField("StdDev");
|
||||
TableHeader header = new TableHeader().addTableHeaderField("CommitTime").addTableHeaderField("Min")
|
||||
.addTableHeaderField("10th").addTableHeaderField("50th").addTableHeaderField("avg").addTableHeaderField("95th")
|
||||
.addTableHeaderField("Max").addTableHeaderField("NumFiles").addTableHeaderField("StdDev");
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -48,12 +48,11 @@ public class HiveUtil {
|
||||
ResultSet rs = null;
|
||||
Statement stmt = conn.createStatement();
|
||||
try {
|
||||
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||
// stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat");
|
||||
stmt.execute("set hive.stats.autogather=false");
|
||||
rs = stmt.executeQuery(
|
||||
"select count(`_hoodie_commit_time`) as cnt from " + dbName + "."
|
||||
+ source.getTableConfig().getTableName());
|
||||
"select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source.getTableConfig().getTableName());
|
||||
long count = -1;
|
||||
if (rs.next()) {
|
||||
count = rs.getLong("cnt");
|
||||
@@ -88,7 +87,7 @@ public class HiveUtil {
|
||||
ResultSet rs = null;
|
||||
Statement stmt = conn.createStatement();
|
||||
try {
|
||||
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||
// stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat");
|
||||
stmt.execute("set hive.stats.autogather=false");
|
||||
rs = stmt.executeQuery(
|
||||
|
||||
@@ -40,8 +40,8 @@ public class SparkUtil {
|
||||
public static SparkLauncher initLauncher(String propertiesFile) throws URISyntaxException {
|
||||
String currentJar = new File(SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath())
|
||||
.getAbsolutePath();
|
||||
SparkLauncher sparkLauncher = new SparkLauncher().setAppResource(currentJar)
|
||||
.setMainClass(SparkMain.class.getName());
|
||||
SparkLauncher sparkLauncher =
|
||||
new SparkLauncher().setAppResource(currentJar).setMainClass(SparkMain.class.getName());
|
||||
|
||||
if (!StringUtils.isNullOrEmpty(propertiesFile)) {
|
||||
sparkLauncher.setPropertiesFile(propertiesFile);
|
||||
|
||||
Reference in New Issue
Block a user