1
0

New Features in DeltaStreamer :

(1) Apply transformation when using delta-streamer to ingest data.
 (2) Add Hudi Incremental Source for Delta Streamer
 (3) Allow delta-streamer config-property to be passed as command-line
 (4) Add Hive Integration to Delta-Streamer and address Review comments
 (5) Ensure MultiPartKeysValueExtractor  handle hive style partition description
 (6) Reuse same spark session on both source and transformer
 (7) Support extracting partition fields from _hoodie_partition_path for HoodieIncrSource
 (8) Reuse Binary Avro coders
 (9) Add push down filter for Incremental source
 (10) Add Hoodie DeltaStreamer metrics to track total time taken
This commit is contained in:
Balaji Varadarajan
2018-10-10 10:31:34 -07:00
committed by vinoth chandar
parent c70dbc13e9
commit 3a0044216c
65 changed files with 2752 additions and 911 deletions

View File

@@ -17,6 +17,7 @@
package com.uber.hoodie.cli.commands;
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
import com.uber.hoodie.avro.model.HoodieCommitMetadata;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.TableHeader;
@@ -32,6 +33,7 @@ import java.util.List;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.specific.SpecificData;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.springframework.shell.core.CommandMarker;
@@ -48,6 +50,92 @@ public class ArchivedCommitsCommand implements CommandMarker {
return HoodieCLI.tableMetadata != null;
}
@CliCommand(value = "show archived commit stats", help = "Read commits from archived files and show details")
public String showArchivedCommits(
@CliOption(key = {"archiveFolderPattern"}, help = "Archive Folder", unspecifiedDefaultValue = "") String folder,
@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 {
System.out.println("===============> Showing only " + limit + " archived commits <===============");
String basePath = HoodieCLI.tableMetadata.getBasePath();
Path archivePath = new Path(basePath + "/.hoodie/.commits_.archive*");
if (folder != null && !folder.isEmpty()) {
archivePath = new Path(basePath + "/.hoodie/" + folder);
}
FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
List<Comparable[]> allStats = new ArrayList<>();
for (FileStatus fs : fsStatuses) {
//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
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)
.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"));
final String instantTime = r.get("commitTime").toString();
final String action = r.get("actionType").toString();
return metadata.getPartitionToWriteStats().values().stream().flatMap(hoodieWriteStats -> {
return hoodieWriteStats.stream().map(hoodieWriteStat -> {
List<Comparable> row = new ArrayList<>();
row.add(action);
row.add(instantTime);
row.add(hoodieWriteStat.getPartitionPath());
row.add(hoodieWriteStat.getFileId());
row.add(hoodieWriteStat.getPrevCommit());
row.add(hoodieWriteStat.getNumWrites());
row.add(hoodieWriteStat.getNumInserts());
row.add(hoodieWriteStat.getNumDeletes());
row.add(hoodieWriteStat.getNumUpdateWrites());
row.add(hoodieWriteStat.getTotalLogFiles());
row.add(hoodieWriteStat.getTotalLogBlocks());
row.add(hoodieWriteStat.getTotalCorruptLogBlock());
row.add(hoodieWriteStat.getTotalRollbackBlocks());
row.add(hoodieWriteStat.getTotalLogRecords());
row.add(hoodieWriteStat.getTotalUpdatedRecordsCompacted());
row.add(hoodieWriteStat.getTotalWriteBytes());
row.add(hoodieWriteStat.getTotalWriteErrors());
return row;
});
}).map(rowList -> rowList.toArray(new Comparable[0]));
}).collect(Collectors.toList());
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")
.addTableHeaderField("total_write_errors");
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, allStats);
}
@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")